From 903525a3844c4be38fc17dcb34186557905d8d11 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 16 Oct 2023 17:49:59 +0800 Subject: [PATCH] fix bench_dbms build (#8194) close pingcap/tiflash#7924 --- dbms/src/Flash/tests/bench_exchange.cpp | 464 ------------------ dbms/src/Flash/tests/bench_exchange.h | 279 ----------- dbms/src/Flash/tests/bench_window.cpp | 213 -------- dbms/src/Functions/tests/bench_collation.cpp | 158 ------ .../Functions/tests/bench_function_cast.cpp | 40 +- .../Functions/tests/bench_function_ilike.cpp | 129 +++++ .../Functions/tests/bench_function_least.cpp | 16 +- dbms/src/TestUtils/bench_dbms_main.cpp | 2 + tests/docker/cluster_tidb_fail_point.yaml | 2 +- .../fail-point-tests/resolve-lock.test | 2 +- 10 files changed, 161 insertions(+), 1144 deletions(-) delete mode 100644 dbms/src/Flash/tests/bench_exchange.cpp delete mode 100644 dbms/src/Flash/tests/bench_exchange.h delete mode 100644 dbms/src/Flash/tests/bench_window.cpp delete mode 100644 dbms/src/Functions/tests/bench_collation.cpp diff --git a/dbms/src/Flash/tests/bench_exchange.cpp b/dbms/src/Flash/tests/bench_exchange.cpp deleted file mode 100644 index 72b4dad2df0..00000000000 --- a/dbms/src/Flash/tests/bench_exchange.cpp +++ /dev/null @@ -1,464 +0,0 @@ -// Copyright 2023 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 // to include the implementation of ExchangeReceiver -#include // to include the implementation of FineGrainedShuffleWriter -#include // to include the implementation of HashParitionWriter -#include // to include the implementation of MPPTunnel -#include // to include the implementation of MPPTunnelSet -#include -#include - - -namespace DB -{ -namespace tests -{ -std::random_device rd; - -MockBlockInputStream::MockBlockInputStream(const std::vector & blocks_, StopFlag & stop_flag_) - : blocks(blocks_) - , header(blocks[0].cloneEmpty()) - , mt(rd()) - , dist(0, blocks.size() - 1) - , stop_flag(stop_flag_) -{} - -MockFixedRowsBlockInputStream::MockFixedRowsBlockInputStream(size_t total_rows_, const std::vector & blocks_) - : header(blocks_[0].cloneEmpty()) - , mt(rd()) - , dist(0, blocks_.size() - 1) - , current_rows(0) - , total_rows(total_rows_) - , blocks(blocks_) -{} - -Block makeBlock(int row_num, bool skew) -{ - InferredDataVector> int64_vec; - InferredDataVector> int64_vec2; - InferredDataVector> string_vec; - - if (skew) - { - for (int i = 0; i < row_num; ++i) - { - int64_vec.emplace_back(100); - int64_vec2.emplace_back(100); - } - - for (int i = 0; i < row_num; ++i) - { - string_vec.push_back("abcdefg"); - } - } - else - { - std::mt19937 mt(rd()); - std::uniform_int_distribution int64_dist; - std::uniform_int_distribution len_dist(10, 20); - std::uniform_int_distribution char_dist; - - for (int i = 0; i < row_num; ++i) - { - int64_vec.emplace_back(int64_dist(mt)); - int64_vec2.emplace_back(int64_dist(mt)); - } - - for (int i = 0; i < row_num; ++i) - { - int len = len_dist(mt); - String s; - for (int j = 0; j < len; ++j) - s.push_back(char_dist(mt)); - string_vec.push_back(std::move(s)); - } - } - - auto int64_data_type = makeDataType>(); - ColumnWithTypeAndName int64_column( - makeColumn>(int64_data_type, int64_vec), - int64_data_type, - "int64_1"); - ColumnWithTypeAndName int64_column2( - makeColumn>(int64_data_type, int64_vec2), - int64_data_type, - "int64_2"); - - auto string_data_type = makeDataType>(); - ColumnWithTypeAndName string_column( - makeColumn>(string_data_type, string_vec), - string_data_type, - "string"); - - return Block({int64_column, string_column, int64_column2}); -} - -std::vector makeBlocks(int block_num, int row_num, bool skew) -{ - std::vector blocks; - for (int i = 0; i < block_num; ++i) - blocks.push_back(makeBlock(row_num, skew)); - return blocks; -} - -mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num) -{ - auto block = makeBlock(row_num); - codec.encode(block, 0, row_num); - - mpp::MPPDataPacket packet; - packet.add_chunks(codec.getString()); - codec.clear(); - - return packet; -} - -std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num) -{ - std::vector packets; - for (int i = 0; i < packet_num; ++i) - packets.push_back(std::make_shared(makePacket(codec, row_num))); - return packets; -} - -std::vector makePacketQueues(int source_num, int queue_size) -{ - std::vector queues(source_num); - for (int i = 0; i < source_num; ++i) - queues[i] = std::make_shared(queue_size); - return queues; -} - -std::vector makeFields() -{ - std::vector fields(3); - fields[0].set_tp(TiDB::TypeLongLong); - fields[1].set_tp(TiDB::TypeString); - fields[2].set_tp(TiDB::TypeLongLong); - return fields; -} - -void printException(const Exception & e) -{ - std::string text = e.displayText(); - - auto embedded_stack_trace_pos = text.find("Stack trace"); - std::cerr << "Code: " << e.code() << ". " << text << std::endl << std::endl; - if (std::string::npos == embedded_stack_trace_pos) - std::cerr << "Stack trace:" << std::endl << e.getStackTrace().toString() << std::endl; -} - -ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_grained_shuffle_stream_count_) - : concurrency(concurrency_) - , source_num(source_num_) - , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) -{ - pb_exchange_receiver.set_tp(tipb::Hash); - for (int i = 0; i < source_num; ++i) - { - mpp::TaskMeta task; - task.set_start_ts(0); - task.set_query_ts(i); - task.set_server_id(i); - task.set_local_query_id(i); - task.set_task_id(i); - task.set_partition_id(i); - task.set_address(""); - - String encoded_task; - task.SerializeToString(&encoded_task); - - pb_exchange_receiver.add_encoded_task_meta(encoded_task); - } - - fields = makeFields(); - *pb_exchange_receiver.add_field_types() = fields[0]; - *pb_exchange_receiver.add_field_types() = fields[1]; - *pb_exchange_receiver.add_field_types() = fields[2]; - - task_meta.set_task_id(100); - - queues = makePacketQueues(source_num, 10); -} - -MockExchangeReceiverPtr ReceiverHelper::buildReceiver() -{ - return std::make_shared( - std::make_shared(queues, fields), - source_num, - concurrency, - "mock_req_id", - "mock_exchange_receiver_id", - fine_grained_shuffle_stream_count); -} - -std::vector ReceiverHelper::buildExchangeReceiverStream() -{ - auto receiver = buildReceiver(); - std::vector streams(concurrency); - // NOTE: check if need fine_grained_shuffle_stream_count - for (int i = 0; i < concurrency; ++i) - { - streams[i] = std::make_shared( - receiver, - "mock_req_id", - "mock_executor_id" + std::to_string(i), - /*stream_id=*/enableFineGrainedShuffle(fine_grained_shuffle_stream_count) ? i : 0); - } - return streams; -} - -BlockInputStreamPtr ReceiverHelper::buildUnionStream() -{ - auto streams = buildExchangeReceiverStream(); - return std::make_shared>(streams, BlockInputStreams{}, concurrency, /*req_id=*/""); -} - -void ReceiverHelper::finish() -{ - if (join_ptr) - { - join_ptr->setBuildTableState(Join::BuildTableState::SUCCEED); - std::cout << fmt::format("Hash table size: {} bytes", join_ptr->getTotalByteCount()) << std::endl; - } -} - -SenderHelper::SenderHelper( - int source_num_, - int concurrency_, - uint32_t fine_grained_shuffle_stream_count_, - int64_t fine_grained_shuffle_batch_size_, - const std::vector & queues_, - const std::vector & fields) - : source_num(source_num_) - , concurrency(concurrency_) - , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) - , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) - , queues(queues_) -{ - mpp::TaskMeta task_meta; - tunnel_set = std::make_shared("mock_req_id"); - for (int i = 0; i < source_num; ++i) - { - auto writer = std::make_shared(queues[i]); - mock_writers.push_back(writer); - - auto tunnel = std::make_shared( - task_meta, - task_meta, - std::chrono::seconds(60), - concurrency, - false, - false, - "mock_req_id"); - tunnel->connect(writer.get()); - tunnels.push_back(tunnel); - MPPTaskId id(0, i); - tunnel_set->registerTunnel(id, tunnel); - } - - tipb::DAGRequest dag_request; - tipb::Executor root_executor; - root_executor.set_executor_id("ExchangeSender_100"); - *dag_request.mutable_root_executor() = root_executor; - - dag_context = std::make_unique(dag_request); - dag_context->is_mpp_task = true; - dag_context->is_root_mpp_task = false; - dag_context->encode_type = tipb::EncodeType::TypeCHBlock; - dag_context->result_field_types = fields; -} - -BlockInputStreamPtr SenderHelper::buildUnionStream(StopFlag & stop_flag, const std::vector & blocks) -{ - std::vector send_streams; - for (int i = 0; i < concurrency; ++i) - { - BlockInputStreamPtr stream = std::make_shared(blocks, stop_flag); - if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) - { - std::unique_ptr response_writer(new FineGrainedShuffleWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - true, - *dag_context, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size)); - send_streams.push_back( - std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); - } - else - { - std::unique_ptr response_writer(new HashParitionWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - -1, - true, - *dag_context)); - send_streams.push_back( - std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); - } - } - - return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); -} - -BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std::vector & blocks) -{ - std::vector send_streams; - for (int i = 0; i < concurrency; ++i) - { - BlockInputStreamPtr stream = std::make_shared(total_rows / concurrency, blocks); - if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) - { - std::unique_ptr response_writer(new FineGrainedShuffleWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - true, - *dag_context, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size)); - send_streams.push_back( - std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); - } - else - { - std::unique_ptr response_writer(new HashParitionWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - -1, - true, - *dag_context)); - send_streams.push_back( - std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); - } - } - - return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); -} - -void SenderHelper::finish() -{ - for (size_t i = 0; i < tunnels.size(); ++i) - { - tunnels[i]->writeDone(); - tunnels[i]->waitForFinish(); - mock_writers[i]->finish(); - } -} - -void ExchangeBench::SetUp(const benchmark::State &) -{ - DynamicThreadPool::global_instance = std::make_unique( - /*fixed_thread_num=*/300, - std::chrono::milliseconds(100000)); - - uniform_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024); - skew_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024, /*skew=*/true); - - try - { - DB::registerWindowFunctions(); - DB::registerFunctions(); - } - catch (DB::Exception &) - { - // Maybe another test has already registered, ignore exception here. - } -} - -void ExchangeBench::TearDown(const benchmark::State &) -{ - uniform_blocks.clear(); - skew_blocks.clear(); - // NOTE: Must reset here, otherwise DynamicThreadPool::fixedWork() may core because metrics already destroyed. - DynamicThreadPool::global_instance.reset(); -} - -void ExchangeBench::runAndWait( - std::shared_ptr receiver_helper, - BlockInputStreamPtr receiver_stream, - std::shared_ptr & sender_helper, - BlockInputStreamPtr sender_stream) -{ - std::future sender_future - = DynamicThreadPool::global_instance->schedule(/*memory_tracker=*/false, [sender_stream, sender_helper] { - sender_stream->readPrefix(); - while (const auto & block = sender_stream->read()) {} - sender_stream->readSuffix(); - sender_helper->finish(); - }); - std::future receiver_future - = DynamicThreadPool::global_instance->schedule(/*memory_tracker=*/false, [receiver_stream, receiver_helper] { - receiver_stream->readPrefix(); - while (const auto & block = receiver_stream->read()) {} - receiver_stream->readSuffix(); - receiver_helper->finish(); - }); - sender_future.get(); - receiver_future.get(); -} - -BENCHMARK_DEFINE_F(ExchangeBench, basic_send_receive) -(benchmark::State & state) -try -{ - const int concurrency = state.range(0); - const int source_num = state.range(1); - const int total_rows = state.range(2); - const int fine_grained_shuffle_stream_count = state.range(3); - const int fine_grained_shuffle_batch_size = state.range(4); - Context context = TiFlashTestEnv::getContext(); - - for (auto _ : state) - { - std::shared_ptr receiver_helper - = std::make_shared(concurrency, source_num, fine_grained_shuffle_stream_count); - BlockInputStreamPtr receiver_stream = receiver_helper->buildUnionStream(); - - std::shared_ptr sender_helper = std::make_shared( - source_num, - concurrency, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size, - receiver_helper->queues, - receiver_helper->fields); - BlockInputStreamPtr sender_stream = sender_helper->buildUnionStream(total_rows, uniform_blocks); - - runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); - } -} -CATCH -BENCHMARK_REGISTER_F(ExchangeBench, basic_send_receive) - ->Args({8, 1, 1024 * 1000, 0, 4096}) - ->Args({8, 1, 1024 * 1000, 4, 4096}) - ->Args({8, 1, 1024 * 1000, 8, 4096}) - ->Args({8, 1, 1024 * 1000, 16, 4096}) - ->Args({8, 1, 1024 * 1000, 32, 4096}) - ->Args({8, 1, 1024 * 1000, 8, 1}) - ->Args({8, 1, 1024 * 1000, 8, 1000}) - ->Args({8, 1, 1024 * 1000, 8, 10000}) - ->Args({8, 1, 1024 * 1000, 8, 100000}); - - -} // namespace tests -} // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.h b/dbms/src/Flash/tests/bench_exchange.h deleted file mode 100644 index 51539f83c97..00000000000 --- a/dbms/src/Flash/tests/bench_exchange.h +++ /dev/null @@ -1,279 +0,0 @@ -// Copyright 2023 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 -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -namespace DB -{ -namespace tests -{ - - -using Packet = mpp::MPPDataPacket; -using PacketPtr = std::shared_ptr; -using PacketQueue = MPMCQueue; -using PacketQueuePtr = std::shared_ptr; -using StopFlag = std::atomic; - -// NOLINTBEGIN(readability-convert-member-functions-to-static) -struct MockReceiverContext -{ - using Status = ::grpc::Status; - struct Request - { - String debugString() const { return "{Request}"; } - - int source_index = 0; - int send_task_id = 0; - int recv_task_id = -1; - }; - - struct Reader - { - explicit Reader(const PacketQueuePtr & queue_) - : queue(queue_) - {} - - void initialize() const {} - - bool read(PacketPtr & packet [[maybe_unused]]) const - { - PacketPtr res; - if (queue->pop(res)) - { - *packet = *res; // avoid change shared packets - return true; - } - return false; - } - - Status finish() const { return ::grpc::Status(); } - - PacketQueuePtr queue; - }; - - struct MockAsyncGrpcExchangePacketReader - { - // Not implement benchmark for Async GRPC for now. - void init(GRPCKickTag *) { assert(0); } - void read(TrackedMppDataPacketPtr &, GRPCKickTag *) { assert(0); } - void finish(::grpc::Status &, GRPCKickTag *) { assert(0); } - }; - - using AsyncReader = MockAsyncGrpcExchangePacketReader; - - MockReceiverContext(const std::vector & queues_, const std::vector & field_types_) - : queues(queues_) - , field_types(field_types_) - {} - - void fillSchema(DAGSchema & schema) const - { - schema.clear(); - for (size_t i = 0; i < field_types.size(); ++i) - { - String name = "exchange_receiver_" + std::to_string(i); - ColumnInfo info = TiDB::fieldTypeToColumnInfo(field_types[i]); - schema.emplace_back(std::move(name), std::move(info)); - } - } - - Request makeRequest(int index) const { return {index, index, -1}; } - - std::shared_ptr makeSyncReader(const Request & request) - { - return std::make_shared(queues[request.send_task_id]); - } - - static Status getStatusOK() { return ::grpc::Status(); } - - bool supportAsync(const Request &) const { return false; } - std::unique_ptr makeAsyncReader(const Request &, grpc::CompletionQueue *, GRPCKickTag *) const - { - return nullptr; - } - - std::vector queues; - std::vector field_types; -}; -// NOLINTEND(readability-convert-member-functions-to-static) - -using MockExchangeReceiver = ExchangeReceiverBase; -using MockExchangeReceiverPtr = std::shared_ptr; -using MockExchangeReceiverInputStream = TiRemoteBlockInputStream; - -struct MockWriter : public PacketWriter -{ - explicit MockWriter(PacketQueuePtr queue_) - : queue(std::move(queue_)) - {} - - bool write(const Packet & packet) override - { - queue->push(std::make_shared(packet)); - return true; - } - - void finish() { queue->finish(); } - - PacketQueuePtr queue; -}; - -using MockWriterPtr = std::shared_ptr; -using MockTunnel = MPPTunnelBase; -using MockTunnelPtr = std::shared_ptr; -using MockTunnelSet = MPPTunnelSetBase; -using MockTunnelSetPtr = std::shared_ptr; - -struct MockBlockInputStream : public IProfilingBlockInputStream -{ - const std::vector & blocks; - Block header; - std::mt19937 mt; - std::uniform_int_distribution dist; - StopFlag & stop_flag; - - MockBlockInputStream(const std::vector & blocks_, StopFlag & stop_flag_); - - String getName() const override { return "MockBlockInputStream"; } - Block getHeader() const override { return header; } - - Block readImpl() override - { - if (stop_flag.load(std::memory_order_relaxed)) - return Block{}; - return blocks[dist(mt)]; - } -}; - -// Similar to MockBlockInputStream, but return fixed count of rows. -struct MockFixedRowsBlockInputStream : public IProfilingBlockInputStream -{ - Block header; - std::mt19937 mt; - std::uniform_int_distribution dist; - size_t current_rows; - size_t total_rows; - const std::vector & blocks; - - MockFixedRowsBlockInputStream(size_t total_rows_, const std::vector & blocks_); - - String getName() const override { return "MockBlockInputStream"; } - Block getHeader() const override { return header; } - - Block readImpl() override - { - if (current_rows >= total_rows) - return Block{}; - Block res = blocks[dist(mt)]; - current_rows += res.rows(); - return res; - } -}; - -Block makeBlock(int row_num, bool skew = false); -std::vector makeBlocks(int block_num, int row_num, bool skew = false); -mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num); -std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num); -std::vector makePacketQueues(int source_num, int queue_size); -std::vector makeFields(); -void printException(const Exception & e); -void sendPacket(const std::vector & packets, const PacketQueuePtr & queue, StopFlag & stop_flag); -void receivePacket(const PacketQueuePtr & queue); - -struct ReceiverHelper -{ - const int concurrency; - const int source_num; - const uint32_t fine_grained_shuffle_stream_count; - tipb::ExchangeReceiver pb_exchange_receiver; - std::vector fields; - mpp::TaskMeta task_meta; - std::vector queues; - std::shared_ptr join_ptr; - - explicit ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_grained_shuffle_stream_count_); - MockExchangeReceiverPtr buildReceiver(); - std::vector buildExchangeReceiverStream(); - BlockInputStreamPtr buildUnionStream(); - void finish(); -}; - -struct SenderHelper -{ - const int source_num; - const int concurrency; - const uint32_t fine_grained_shuffle_stream_count; - const int64_t fine_grained_shuffle_batch_size; - - std::vector queues; - std::vector mock_writers; - std::vector tunnels; - MockTunnelSetPtr tunnel_set; - std::unique_ptr dag_context; - - SenderHelper( - int source_num_, - int concurrency_, - uint32_t fine_grained_shuffle_stream_count_, - int64_t fine_grained_shuffle_batch_size_, - const std::vector & queues_, - const std::vector & fields); - - // Using MockBlockInputStream to build streams. - BlockInputStreamPtr buildUnionStream(StopFlag & stop_flag, const std::vector & blocks); - // Using MockFixedRowsBlockInputStream to build streams. - BlockInputStreamPtr buildUnionStream(size_t total_rows, const std::vector & blocks); - - void finish(); -}; - -class ExchangeBench : public benchmark::Fixture -{ -public: - void SetUp(const benchmark::State &) override; - void TearDown(const benchmark::State &) override; - void runAndWait( - std::shared_ptr receiver_helper, - BlockInputStreamPtr receiver_stream, - std::shared_ptr & sender_helper, - BlockInputStreamPtr sender_stream); - - std::vector uniform_blocks; - std::vector skew_blocks; -}; - - -} // namespace tests -} // namespace DB diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp deleted file mode 100644 index 1ba0bfd2ed4..00000000000 --- a/dbms/src/Flash/tests/bench_window.cpp +++ /dev/null @@ -1,213 +0,0 @@ -// Copyright 2023 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 -{ -namespace tests -{ -class WindowFunctionBench : public ExchangeBench -{ -public: - void SetUp(const benchmark::State & state) override - { - // Using DAGRequestBuilder to build tipb::Window and tipb::Sort. - // select row_number() over w1 from t1 window w1 as (partition by c1, c2, c3 order by c1, c2, c3); - ExchangeBench::SetUp(state); - } - - static void setupPB(uint64_t fine_grained_shuffle_stream_count, tipb::Window & window, tipb::Sort & sort) - { - MockColumnInfoVec columns{ - {"c1", TiDB::TP::TypeLongLong}, - {"c2", TiDB::TP::TypeString}, - {"c3", TiDB::TP::TypeLongLong}, - }; - size_t executor_index = 0; - DAGRequestBuilder builder(executor_index); - builder.mockTable("test", "t1", 0 /*table_id=*/, columns) - .sort({{"c1", false}, {"c2", false}, {"c3", false}}, true, fine_grained_shuffle_stream_count) - .window( - RowNumber(), - {{"c1", false}, {"c2", false}, {"c3", false}}, - {{"c1", false}, {"c2", false}, {"c3", false}}, - buildDefaultRowsFrame(), - fine_grained_shuffle_stream_count); - tipb::DAGRequest req; - MPPInfo mpp_info(0, 0, 0, 0, -1, -1, {}, std::unordered_map>{}); - builder.getRoot() - ->toTiPBExecutor(req.mutable_root_executor(), /*collator_id=*/0, mpp_info, TiFlashTestEnv::getContext()); - assert(req.root_executor().tp() == tipb::TypeWindow); - window = req.root_executor().window(); - assert(window.child().tp() == tipb::TypeSort); - sort = window.child().sort(); - } - - static void prepareWindowStream( - Context & context, - int concurrency, - int source_num, - int total_rows, - uint32_t fine_grained_shuffle_stream_count, - uint64_t fine_grained_shuffle_batch_size, - const std::vector & blocks, - BlockInputStreamPtr & sender_stream, - BlockInputStreamPtr & receiver_stream, - std::shared_ptr & sender_helper, - std::shared_ptr & receiver_helper, - bool build_window = true) - { - tipb::Window window; - tipb::Sort sort; - setupPB(fine_grained_shuffle_stream_count, window, sort); - - DAGPipeline pipeline; - receiver_helper = std::make_shared(concurrency, source_num, fine_grained_shuffle_stream_count); - pipeline.streams = receiver_helper->buildExchangeReceiverStream(); - - sender_helper = std::make_shared( - source_num, - concurrency, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size, - receiver_helper->queues, - receiver_helper->fields); - sender_stream = sender_helper->buildUnionStream(total_rows, blocks); - - context.setDAGContext(sender_helper->dag_context.get()); - std::vector source_columns{ - NameAndTypePair("c1", makeNullable(std::make_shared())), - NameAndTypePair("c2", makeNullable(std::make_shared())), - NameAndTypePair("c3", makeNullable(std::make_shared()))}; - auto mock_interpreter = mockInterpreter(context, source_columns, concurrency); - mock_interpreter->input_streams_vec.push_back(pipeline.streams); - mockExecuteWindowOrder(mock_interpreter, pipeline, sort, fine_grained_shuffle_stream_count); - if (build_window) - { - mockExecuteWindow(mock_interpreter, pipeline, window, fine_grained_shuffle_stream_count); - } - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, 8192, 0, "mock_executor_id_squashing"); - }); - receiver_stream = std::make_shared>( - pipeline.streams, - BlockInputStreams{}, - concurrency, - /*req_id=*/""); - } -}; - -BENCHMARK_DEFINE_F(WindowFunctionBench, basic_row_number) -(benchmark::State & state) -try -{ - const int concurrency = state.range(0); - const int source_num = state.range(1); - const int total_rows = state.range(2); - const int fine_grained_shuffle_stream_count = state.range(3); - const int fine_grained_shuffle_batch_size = state.range(4); - const bool skew = state.range(5); - Context context = TiFlashTestEnv::getContext(); - - std::vector * blocks = &uniform_blocks; - if (skew) - blocks = &skew_blocks; - - for (auto _ : state) - { - std::shared_ptr sender_helper; - std::shared_ptr receiver_helper; - BlockInputStreamPtr sender_stream; - BlockInputStreamPtr receiver_stream; - - prepareWindowStream( - context, - concurrency, - source_num, - total_rows, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size, - *blocks, - sender_stream, - receiver_stream, - sender_helper, - receiver_helper); - - runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); - } -} -CATCH -BENCHMARK_REGISTER_F(WindowFunctionBench, basic_row_number) - ->Args({8, 1, 1024 * 1000, 0, 4096, false}) // Test fine_grained_shuffle_stream_count. - ->Args({8, 1, 1024 * 1000, 4, 4096, false}) - ->Args({8, 1, 1024 * 1000, 8, 4096, false}) - ->Args({8, 1, 1024 * 1000, 16, 4096, false}) - ->Args({8, 1, 1024 * 1000, 32, 4096, false}) - ->Args({8, 1, 1024 * 1000, 8, 1, false}) // Test fine_grained_shuffle_batch_size. - ->Args({8, 1, 1024 * 1000, 8, 1000, false}) - ->Args({8, 1, 1024 * 1000, 8, 10000, false}) - ->Args({8, 1, 1024 * 1000, 8, 100000, false}) - ->Args({8, 1, 1024 * 1000, 0, 4096, true}) // Test skew dataset. - ->Args({8, 1, 1024 * 1000, 4, 4096, true}) - ->Args({8, 1, 1024 * 1000, 8, 4096, true}) - ->Args({8, 1, 1024 * 1000, 16, 4096, true}); - -BENCHMARK_DEFINE_F(WindowFunctionBench, partial_sort_skew_dataset) -(benchmark::State & state) -try -{ - const int concurrency = state.range(0); - const int source_num = state.range(1); - const int total_rows = state.range(2); - const int fine_grained_shuffle_stream_count = state.range(3); - const int fine_grained_shuffle_batch_size = state.range(4); - Context context = TiFlashTestEnv::getContext(); - - std::vector * blocks = &skew_blocks; - - for (auto _ : state) - { - std::shared_ptr sender_helper; - std::shared_ptr receiver_helper; - BlockInputStreamPtr sender_stream; - BlockInputStreamPtr receiver_stream; - - // Only build partial sort. - prepareWindowStream( - context, - concurrency, - source_num, - total_rows, - fine_grained_shuffle_stream_count, - fine_grained_shuffle_batch_size, - *blocks, - sender_stream, - receiver_stream, - sender_helper, - receiver_helper, - /*build_window=*/false); - - runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); - } -} -CATCH -BENCHMARK_REGISTER_F(WindowFunctionBench, partial_sort_skew_dataset) - ->Args({1, 1, 1024 * 10000, 0, 4096}) // Test how much multiple-thread improves performance for partial sort. - ->Args({2, 1, 1024 * 10000, 0, 4096}) - ->Args({4, 1, 1024 * 10000, 0, 4096}) - ->Args({8, 1, 1024 * 10000, 0, 4096}); -} // namespace tests -} // namespace DB diff --git a/dbms/src/Functions/tests/bench_collation.cpp b/dbms/src/Functions/tests/bench_collation.cpp deleted file mode 100644 index ed34c52e023..00000000000 --- a/dbms/src/Functions/tests/bench_collation.cpp +++ /dev/null @@ -1,158 +0,0 @@ -// Copyright 2023 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 - -/// this is a hack, include the cpp file so we can test MatchImpl directly -#include // NOLINT - -namespace DB -{ -namespace tests -{ - -class CollationBench : public benchmark::Fixture -{ -public: - using ColStringType = typename TypeTraits::FieldType; - using ColUInt8Type = typename TypeTraits::FieldType; - - ColumnsWithTypeAndName data{ - toVec("col0", std::vector(1000000, "aaaaaaaaaaaaa")), - toVec("col1", std::vector(1000000, "aaaaaaaaaaaaa")), - toVec("result", std::vector{})}; - - ColumnsWithTypeAndName like_data{ - toVec("col0", std::vector(1000000, "qwdgefwabchfue")), - createConstColumn(1000000, "%abc%"), - createConstColumn(1000000, static_cast('\\')), - toVec("result", std::vector{})}; -}; - -class CollationLessBench : public CollationBench -{ -public: - void SetUp(const benchmark::State &) override {} -}; - -class CollationEqBench : public CollationBench -{ -public: - void SetUp(const benchmark::State &) override {} -}; - -class CollationLikeBench : public CollationBench -{ -public: - void SetUp(const benchmark::State &) override {} -}; - -#define BENCH_LESS_COLLATOR(collator) \ - BENCHMARK_DEFINE_F(CollationLessBench, collator) \ - (benchmark::State & state) \ - try \ - { \ - FunctionLess fl; \ - TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ - fl.setCollator(collator); \ - Block block(data); \ - ColumnNumbers arguments{0, 1}; \ - for (auto _ : state) \ - { \ - fl.executeImpl(block, arguments, 2); \ - } \ - } \ - CATCH \ - BENCHMARK_REGISTER_F(CollationLessBench, collator)->Iterations(10); - - -#define BENCH_EQ_COLLATOR(collator) \ - BENCHMARK_DEFINE_F(CollationEqBench, collator) \ - (benchmark::State & state) \ - try \ - { \ - FunctionEquals fe; \ - TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ - fe.setCollator(collator); \ - Block block(data); \ - ColumnNumbers arguments{0, 1}; \ - for (auto _ : state) \ - { \ - fe.executeImpl(block, arguments, 2); \ - } \ - } \ - CATCH \ - BENCHMARK_REGISTER_F(CollationEqBench, collator)->Iterations(10); - - -#define BENCH_LIKE_COLLATOR(collator) \ - BENCHMARK_DEFINE_F(CollationLikeBench, collator) \ - (benchmark::State & state) \ - try \ - { \ - FunctionLike3Args fl; \ - TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ - fl.setCollator(collator); \ - Block block(like_data); \ - ColumnNumbers arguments{0, 1, 2}; \ - for (auto _ : state) \ - { \ - fl.executeImpl(block, arguments, 3); \ - } \ - } \ - CATCH \ - BENCHMARK_REGISTER_F(CollationLikeBench, collator)->Iterations(10); - - -BENCH_LESS_COLLATOR(UTF8MB4_BIN); -BENCH_LESS_COLLATOR(UTF8MB4_GENERAL_CI); -BENCH_LESS_COLLATOR(UTF8MB4_UNICODE_CI); -BENCH_LESS_COLLATOR(UTF8MB4_0900_AI_CI); -BENCH_LESS_COLLATOR(UTF8MB4_0900_BIN); -BENCH_LESS_COLLATOR(UTF8_BIN); -BENCH_LESS_COLLATOR(UTF8_GENERAL_CI); -BENCH_LESS_COLLATOR(UTF8_UNICODE_CI); -BENCH_LESS_COLLATOR(ASCII_BIN); -BENCH_LESS_COLLATOR(BINARY); -BENCH_LESS_COLLATOR(LATIN1_BIN); - -BENCH_EQ_COLLATOR(UTF8MB4_BIN); -BENCH_EQ_COLLATOR(UTF8MB4_GENERAL_CI); -BENCH_EQ_COLLATOR(UTF8MB4_UNICODE_CI); -BENCH_EQ_COLLATOR(UTF8MB4_0900_AI_CI); -BENCH_EQ_COLLATOR(UTF8MB4_0900_BIN); -BENCH_EQ_COLLATOR(UTF8_BIN); -BENCH_EQ_COLLATOR(UTF8_GENERAL_CI); -BENCH_EQ_COLLATOR(UTF8_UNICODE_CI); -BENCH_EQ_COLLATOR(ASCII_BIN); -BENCH_EQ_COLLATOR(BINARY); -BENCH_EQ_COLLATOR(LATIN1_BIN); - -BENCH_LIKE_COLLATOR(UTF8MB4_BIN); -BENCH_LIKE_COLLATOR(UTF8MB4_GENERAL_CI); -BENCH_LIKE_COLLATOR(UTF8MB4_UNICODE_CI); -BENCH_LIKE_COLLATOR(UTF8MB4_0900_AI_CI); -BENCH_LIKE_COLLATOR(UTF8MB4_0900_BIN); -BENCH_LIKE_COLLATOR(UTF8_BIN); -BENCH_LIKE_COLLATOR(UTF8_GENERAL_CI); -BENCH_LIKE_COLLATOR(UTF8_UNICODE_CI); -BENCH_LIKE_COLLATOR(ASCII_BIN); -BENCH_LIKE_COLLATOR(BINARY); -BENCH_LIKE_COLLATOR(LATIN1_BIN); - -} // namespace tests -} // namespace DB diff --git a/dbms/src/Functions/tests/bench_function_cast.cpp b/dbms/src/Functions/tests/bench_function_cast.cpp index 4b8a8399b65..a276bb7e197 100644 --- a/dbms/src/Functions/tests/bench_function_cast.cpp +++ b/dbms/src/Functions/tests/bench_function_cast.cpp @@ -355,26 +355,26 @@ class CastToDecimalBench : public benchmark::Fixture std::vector dest_float64_vec; }; -#define CAST_BENCHMARK(CLASS_NAME, CASE_NAME, FROM_COL, DEST_TYPE) \ - BENCHMARK_DEFINE_F(CLASS_NAME, CASE_NAME) \ - (benchmark::State & state) \ - try \ - { \ - const String func_name = "tidb_cast"; \ - auto context = DB::tests::TiFlashTestEnv::getContext(); \ - auto dag_context_ptr = std::make_unique(1024); \ - UInt64 ori_flags = dag_context_ptr->getFlags(); \ - dag_context_ptr->addFlag(TiDBSQLFlags::OVERFLOW_AS_WARNING); \ - dag_context_ptr->addFlag(TiDBSQLFlags::TRUNCATE_AS_WARNING); \ - dag_context_ptr->clearWarnings(); \ - context.setDAGContext(dag_context_ptr.get()); \ - for (auto _ : state) \ - { \ - executeFunction(context, func_name, FROM_COL, DEST_TYPE); \ - } \ - dag_context_ptr->setFlags(ori_flags); \ - } \ - CATCH \ +#define CAST_BENCHMARK(CLASS_NAME, CASE_NAME, FROM_COL, DEST_TYPE) \ + BENCHMARK_DEFINE_F(CLASS_NAME, CASE_NAME) \ + (benchmark::State & state) \ + try \ + { \ + const String func_name = "tidb_cast"; \ + auto context = DB::tests::TiFlashTestEnv::getContext(); \ + auto dag_context_ptr = std::make_unique(1024); \ + UInt64 ori_flags = dag_context_ptr->getFlags(); \ + dag_context_ptr->addFlag(TiDBSQLFlags::OVERFLOW_AS_WARNING); \ + dag_context_ptr->addFlag(TiDBSQLFlags::TRUNCATE_AS_WARNING); \ + dag_context_ptr->clearWarnings(); \ + context->setDAGContext(dag_context_ptr.get()); \ + for (auto _ : state) \ + { \ + executeFunction(*context, func_name, FROM_COL, DEST_TYPE); \ + } \ + dag_context_ptr->setFlags(ori_flags); \ + } \ + CATCH \ BENCHMARK_REGISTER_F(CLASS_NAME, CASE_NAME)->Iterations(1000); // NOTE: There are three factors that will affects performance: diff --git a/dbms/src/Functions/tests/bench_function_ilike.cpp b/dbms/src/Functions/tests/bench_function_ilike.cpp index 6df135d1e11..b8125146b5e 100644 --- a/dbms/src/Functions/tests/bench_function_ilike.cpp +++ b/dbms/src/Functions/tests/bench_function_ilike.cpp @@ -177,5 +177,134 @@ try CATCH BENCHMARK_REGISTER_F(LikeBench, like)->Iterations(10); +class CollationBench : public benchmark::Fixture +{ +public: + using ColStringType = typename TypeTraits::FieldType; + using ColUInt8Type = typename TypeTraits::FieldType; + + ColumnsWithTypeAndName data{ + toVec("col0", std::vector(1000000, "aaaaaaaaaaaaa")), + toVec("col1", std::vector(1000000, "aaaaaaaaaaaaa")), + toVec("result", std::vector{})}; + + ColumnsWithTypeAndName like_data{ + toVec("col0", std::vector(1000000, "qwdgefwabchfue")), + createConstColumn(1000000, "%abc%"), + createConstColumn(1000000, static_cast('\\')), + toVec("result", std::vector{})}; +}; + +class CollationLessBench : public CollationBench +{ +public: + void SetUp(const benchmark::State &) override {} +}; + +class CollationEqBench : public CollationBench +{ +public: + void SetUp(const benchmark::State &) override {} +}; + +class CollationLikeBench : public CollationBench +{ +public: + void SetUp(const benchmark::State &) override {} +}; + +#define BENCH_LESS_COLLATOR(collator) \ + BENCHMARK_DEFINE_F(CollationLessBench, collator) \ + (benchmark::State & state) \ + try \ + { \ + FunctionLess fl; \ + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ + fl.setCollator(collator); \ + Block block(data); \ + ColumnNumbers arguments{0, 1}; \ + for (auto _ : state) \ + { \ + fl.executeImpl(block, arguments, 2); \ + } \ + } \ + CATCH \ + BENCHMARK_REGISTER_F(CollationLessBench, collator)->Iterations(10); + + +#define BENCH_EQ_COLLATOR(collator) \ + BENCHMARK_DEFINE_F(CollationEqBench, collator) \ + (benchmark::State & state) \ + try \ + { \ + FunctionEquals fe; \ + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ + fe.setCollator(collator); \ + Block block(data); \ + ColumnNumbers arguments{0, 1}; \ + for (auto _ : state) \ + { \ + fe.executeImpl(block, arguments, 2); \ + } \ + } \ + CATCH \ + BENCHMARK_REGISTER_F(CollationEqBench, collator)->Iterations(10); + + +#define BENCH_LIKE_COLLATOR(collator) \ + BENCHMARK_DEFINE_F(CollationLikeBench, collator) \ + (benchmark::State & state) \ + try \ + { \ + FunctionLike3Args fl; \ + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ + fl.setCollator(collator); \ + Block block(like_data); \ + ColumnNumbers arguments{0, 1, 2}; \ + for (auto _ : state) \ + { \ + fl.executeImpl(block, arguments, 3); \ + } \ + } \ + CATCH \ + BENCHMARK_REGISTER_F(CollationLikeBench, collator)->Iterations(10); + + +BENCH_LESS_COLLATOR(UTF8MB4_BIN); +BENCH_LESS_COLLATOR(UTF8MB4_GENERAL_CI); +BENCH_LESS_COLLATOR(UTF8MB4_UNICODE_CI); +BENCH_LESS_COLLATOR(UTF8MB4_0900_AI_CI); +BENCH_LESS_COLLATOR(UTF8MB4_0900_BIN); +BENCH_LESS_COLLATOR(UTF8_BIN); +BENCH_LESS_COLLATOR(UTF8_GENERAL_CI); +BENCH_LESS_COLLATOR(UTF8_UNICODE_CI); +BENCH_LESS_COLLATOR(ASCII_BIN); +BENCH_LESS_COLLATOR(BINARY); +BENCH_LESS_COLLATOR(LATIN1_BIN); + +BENCH_EQ_COLLATOR(UTF8MB4_BIN); +BENCH_EQ_COLLATOR(UTF8MB4_GENERAL_CI); +BENCH_EQ_COLLATOR(UTF8MB4_UNICODE_CI); +BENCH_EQ_COLLATOR(UTF8MB4_0900_AI_CI); +BENCH_EQ_COLLATOR(UTF8MB4_0900_BIN); +BENCH_EQ_COLLATOR(UTF8_BIN); +BENCH_EQ_COLLATOR(UTF8_GENERAL_CI); +BENCH_EQ_COLLATOR(UTF8_UNICODE_CI); +BENCH_EQ_COLLATOR(ASCII_BIN); +BENCH_EQ_COLLATOR(BINARY); +BENCH_EQ_COLLATOR(LATIN1_BIN); + +BENCH_LIKE_COLLATOR(UTF8MB4_BIN); +BENCH_LIKE_COLLATOR(UTF8MB4_GENERAL_CI); +BENCH_LIKE_COLLATOR(UTF8MB4_UNICODE_CI); +BENCH_LIKE_COLLATOR(UTF8MB4_0900_AI_CI); +BENCH_LIKE_COLLATOR(UTF8MB4_0900_BIN); +BENCH_LIKE_COLLATOR(UTF8_BIN); +BENCH_LIKE_COLLATOR(UTF8_GENERAL_CI); +BENCH_LIKE_COLLATOR(UTF8_UNICODE_CI); +BENCH_LIKE_COLLATOR(ASCII_BIN); +BENCH_LIKE_COLLATOR(BINARY); +BENCH_LIKE_COLLATOR(LATIN1_BIN); + } // namespace tests } // namespace DB diff --git a/dbms/src/Functions/tests/bench_function_least.cpp b/dbms/src/Functions/tests/bench_function_least.cpp index f84c57c4123..52854d418f3 100644 --- a/dbms/src/Functions/tests/bench_function_least.cpp +++ b/dbms/src/Functions/tests/bench_function_least.cpp @@ -285,7 +285,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, col1, col2, col3); + executeFunction(*context, func_name, col1, col2, col3); } } CATCH @@ -299,7 +299,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, col1, col2); + executeFunction(*context, func_name, col1, col2); } } CATCH @@ -313,7 +313,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, col_nullable1, col_nullable2, col_nullable3); + executeFunction(*context, func_name, col_nullable1, col_nullable2, col_nullable3); } } CATCH @@ -327,7 +327,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, col1, col2, col3); + executeFunction(*context, func_name, col1, col2, col3); } } CATCH @@ -341,7 +341,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, col1, col2); + executeFunction(*context, func_name, col1, col2); } } CATCH @@ -355,7 +355,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, col_nullable1, col_nullable2, col_nullable3); + executeFunction(*context, func_name, col_nullable1, col_nullable2, col_nullable3); } } CATCH @@ -370,7 +370,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, v_col); + executeFunction(*context, func_name, v_col); } } CATCH @@ -384,7 +384,7 @@ try auto context = DB::tests::TiFlashTestEnv::getContext(); for (auto _ : state) { - executeFunction(context, func_name, v_col); + executeFunction(*context, func_name, v_col); } } CATCH diff --git a/dbms/src/TestUtils/bench_dbms_main.cpp b/dbms/src/TestUtils/bench_dbms_main.cpp index b1e140e9c3b..df19a4691cc 100644 --- a/dbms/src/TestUtils/bench_dbms_main.cpp +++ b/dbms/src/TestUtils/bench_dbms_main.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include @@ -25,6 +26,7 @@ int main(int argc, char * argv[]) DB::tests::TiFlashTestEnv::initializeGlobalContext(); if (::benchmark::ReportUnrecognizedArguments(argc, argv)) return 1; + DB::LocalAdmissionController::global_instance = std::make_unique(); ::benchmark::RunSpecifiedBenchmarks(); DB::tests::TiFlashTestEnv::shutdown(); ::benchmark::Shutdown(); diff --git a/tests/docker/cluster_tidb_fail_point.yaml b/tests/docker/cluster_tidb_fail_point.yaml index 4f399159859..faec3c48898 100644 --- a/tests/docker/cluster_tidb_fail_point.yaml +++ b/tests/docker/cluster_tidb_fail_point.yaml @@ -59,7 +59,7 @@ services: security_opt: - seccomp:unconfined environment: - GO_FAILPOINTS: "github.com/pingcap/tidb/server/enableTestAPI=return" + GO_FAILPOINTS: "github.com/pingcap/tidb/pkg/server/enableTestAPI=return" volumes: - ./config/tidb.toml:/tidb.toml:ro - ./log/tidb0:/log diff --git a/tests/tidb-ci/fail-point-tests/resolve-lock.test b/tests/tidb-ci/fail-point-tests/resolve-lock.test index fdf4ddf1fe7..93daafa8374 100644 --- a/tests/tidb-ci/fail-point-tests/resolve-lock.test +++ b/tests/tidb-ci/fail-point-tests/resolve-lock.test @@ -19,7 +19,7 @@ mysql> alter table test.t set tiflash replica 1; mysql> set @@global.tidb_enable_async_commit=1; set @@global.tidb_allow_batch_cop=0; -curl_tidb> get fail/github.com/pingcap/tidb/server/enableTestAPI +curl_tidb> get fail/github.com/pingcap/tidb/pkg/server/enableTestAPI return curl_tidb> put fail/tikvclient/asyncCommitDoNothing return