diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 7392cf21b41..6686c34679b 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 7392cf21b4163be8c36743641f7c5eee7cf40600 +Subproject commit 6686c34679ba68c58380088a67f5e7c9c70f3a80 diff --git a/contrib/tipb b/contrib/tipb index 3e2483c20a9..d6f1aebebab 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 3e2483c20a9ec3f9ea587c49808c4b66c9716a2c +Subproject commit d6f1aebebabd82bd874c8dcff0b49d694bca0148 diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index a6ba4ea4618..fdcb6a9f2fc 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -97,6 +97,7 @@ add_headers_and_sources(dbms src/Storages/Page/V3) add_headers_and_sources(dbms src/Storages/Page/V3/LogFile) add_headers_and_sources(dbms src/Storages/Page/V3/WAL) add_headers_and_sources(dbms src/Storages/Page/V3/spacemap) +add_headers_and_sources(dbms src/Storages/Page/V3/PageDirectory) add_headers_and_sources(dbms src/Storages/Page/) add_headers_and_sources(dbms src/TiDB) add_headers_and_sources(dbms src/Client) @@ -362,7 +363,11 @@ endif () # dbms add_target_pch("pch-dbms.h" dbms flash_service) -add_target_pch("pch-common.h" clickhouse_common_io clickhouse_functions clickhouse_aggregate_functions) +add_target_pch("pch-common.h" clickhouse_common_io clickhouse_aggregate_functions) + +# disable PCH for `clickhouse_functions` temporarily because some source files need to use specific cxx flags +# add_target_pch("pch-common.h" clickhouse_functions) + add_target_pch("pch-common.h" clickhouse_parsers clickhouse_storages_system dt-workload-lib clickhouse-server-lib) # common diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index da132834d14..5b184ef8983 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/Common/BackgroundTask.cpp b/dbms/src/Common/BackgroundTask.cpp new file mode 100644 index 00000000000..16a23535541 --- /dev/null +++ b/dbms/src/Common/BackgroundTask.cpp @@ -0,0 +1,94 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +{ +bool process_mem_usage(double & resident_set) +{ + resident_set = 0.0; + + // 'file' stat seems to give the most reliable results + std::ifstream stat_stream("/proc/self/stat", std::ios_base::in); + // if "/proc/self/stat" is not supported + if (!stat_stream.is_open()) + return false; + + // dummy vars for leading entries in stat that we don't care about + std::string pid, comm, state, ppid, pgrp, session, tty_nr; + std::string tpgid, flags, minflt, cminflt, majflt, cmajflt; + std::string utime, stime, cutime, cstime, priority, nice; + std::string proc_num_threads, itrealvalue, starttime; + UInt64 vsize; + + // the field we want + Int64 rss; + + stat_stream >> pid >> comm >> state >> ppid >> pgrp >> session >> tty_nr + >> tpgid >> flags >> minflt >> cminflt >> majflt >> cmajflt + >> utime >> stime >> cutime >> cstime >> priority >> nice + >> proc_num_threads >> itrealvalue >> starttime >> vsize >> rss; // don't care about the rest + + stat_stream.close(); + + Int64 page_size_kb = sysconf(_SC_PAGE_SIZE) / 1024; // in case x86-64 is configured to use 2MB pages + resident_set = rss * page_size_kb; + return true; +} + +bool isProcStatSupported() +{ + std::ifstream stat_stream("/proc/self/stat", std::ios_base::in); + return stat_stream.is_open(); +} + +void CollectProcInfoBackgroundTask::begin() +{ + std::unique_lock lk(mu); + if (!is_already_begin) + { + if (!isProcStatSupported()) + { + end_fin = true; + return; + } + std::thread t = ThreadFactory::newThread(false, "MemTrackThread", &CollectProcInfoBackgroundTask::memCheckJob, this); + t.detach(); + is_already_begin = true; + } +} + +void CollectProcInfoBackgroundTask::memCheckJob() +{ + double resident_set; + while (!end_syn) + { + process_mem_usage(resident_set); + resident_set *= 1024; // unit: byte + real_rss = static_cast(resident_set); + + usleep(100000); // sleep 100ms + } + end_fin = true; +} + +void CollectProcInfoBackgroundTask::end() +{ + end_syn = true; + while (!end_fin) + usleep(1000); // Just ok since it is called only when TiFlash shutdown. +} +} // namespace DB diff --git a/dbms/src/Common/BackgroundTask.h b/dbms/src/Common/BackgroundTask.h new file mode 100644 index 00000000000..afeacccd3bc --- /dev/null +++ b/dbms/src/Common/BackgroundTask.h @@ -0,0 +1,39 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 CollectProcInfoBackgroundTask +{ +public: + CollectProcInfoBackgroundTask() = default; + ~CollectProcInfoBackgroundTask() + { + end(); + } + void begin(); + + void end(); + +private: + void memCheckJob(); + + std::mutex mu; + bool is_already_begin = false; + std::atomic end_syn{false}, end_fin{false}; +}; +} // namespace DB diff --git a/dbms/src/Common/MPMCQueue.h b/dbms/src/Common/MPMCQueue.h index ce6d4365393..db82cad0ff2 100644 --- a/dbms/src/Common/MPMCQueue.h +++ b/dbms/src/Common/MPMCQueue.h @@ -81,9 +81,13 @@ class MPMCQueue ~MPMCQueue() { - std::unique_lock lock(mu); - for (; read_pos < write_pos; ++read_pos) - destruct(getObj(read_pos)); + drain(); + } + + void finishAndDrain() + { + finish(); + drain(); } // Cannot to use copy/move constructor, @@ -418,6 +422,13 @@ class MPMCQueue obj.~T(); } + void drain() + { + std::unique_lock lock(mu); + for (; read_pos < write_pos; ++read_pos) + destruct(getObj(read_pos)); + } + template ALWAYS_INLINE bool changeStatus(F && action) { diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index f64881ae35a..da8ecf28e26 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -22,6 +22,7 @@ #include +std::atomic real_rss{0}; MemoryTracker::~MemoryTracker() { if (peak) @@ -46,6 +47,7 @@ MemoryTracker::~MemoryTracker() * then memory usage of 'next' memory trackers will be underestimated, * because amount will be decreased twice (first - here, second - when real 'free' happens). */ + // TODO In future, maybe we can find a better way to handle the "amount > 0" case. if (auto value = amount.load(std::memory_order_relaxed)) free(value); } @@ -80,7 +82,7 @@ void MemoryTracker::alloc(Int64 size, bool check_memory_limit) /// In this case, it doesn't matter. if (unlikely(fault_probability && drand48() < fault_probability)) { - free(size); + amount.fetch_sub(size, std::memory_order_relaxed); DB::FmtBuffer fmt_buf; fmt_buf.append("Memory tracker"); @@ -93,20 +95,33 @@ void MemoryTracker::alloc(Int64 size, bool check_memory_limit) throw DB::TiFlashException(fmt_buf.toString(), DB::Errors::Coprocessor::MemoryLimitExceeded); } - - if (unlikely(current_limit && will_be > current_limit)) + bool is_rss_too_large = (!next.load(std::memory_order_relaxed) && current_limit + && real_rss > current_limit + bytes_rss_larger_than_limit + && will_be > current_limit - (real_rss - current_limit - bytes_rss_larger_than_limit)); + if (is_rss_too_large + || unlikely(current_limit && will_be > current_limit)) { - free(size); + amount.fetch_sub(size, std::memory_order_relaxed); DB::FmtBuffer fmt_buf; fmt_buf.append("Memory limit"); if (description) fmt_buf.fmtAppend(" {}", description); - fmt_buf.fmtAppend(" exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", - formatReadableSizeWithBinarySuffix(will_be), - size, - formatReadableSizeWithBinarySuffix(current_limit)); + if (!is_rss_too_large) + { // out of memory quota + fmt_buf.fmtAppend(" exceeded caused by 'out of memory quota for data computing' : would use {} for data computing (attempt to allocate chunk of {} bytes), limit of memory for data computing: {}", + formatReadableSizeWithBinarySuffix(will_be), + size, + formatReadableSizeWithBinarySuffix(current_limit)); + } + else + { // RSS too large + fmt_buf.fmtAppend(" exceeded caused by 'RSS(Resident Set Size) much larger than limit' : process memory size would be {} for (attempt to allocate chunk of {} bytes), limit of memory for data computing : {}", + formatReadableSizeWithBinarySuffix(real_rss), + size, + formatReadableSizeWithBinarySuffix(current_limit)); + } throw DB::TiFlashException(fmt_buf.toString(), DB::Errors::Coprocessor::MemoryLimitExceeded); } @@ -116,7 +131,17 @@ void MemoryTracker::alloc(Int64 size, bool check_memory_limit) peak.store(will_be, std::memory_order_relaxed); if (auto * loaded_next = next.load(std::memory_order_relaxed)) - loaded_next->alloc(size, check_memory_limit); + { + try + { + loaded_next->alloc(size, check_memory_limit); + } + catch (...) + { + amount.fetch_sub(size, std::memory_order_relaxed); + std::rethrow_exception(std::current_exception()); + } + } } @@ -130,7 +155,7 @@ void MemoryTracker::free(Int64 size) * Memory usage will be calculated with some error. * NOTE The code is not atomic. Not worth to fix. */ - if (new_amount < 0) + if (new_amount < 0 && !next.load(std::memory_order_relaxed)) // handle it only for root memory_tracker { amount.fetch_sub(new_amount); size += new_amount; @@ -170,7 +195,7 @@ thread_local MemoryTracker * current_memory_tracker = nullptr; namespace CurrentMemoryTracker { -static Int64 MEMORY_TRACER_SUBMIT_THRESHOLD = 8 * 1024 * 1024; // 8 MiB +static Int64 MEMORY_TRACER_SUBMIT_THRESHOLD = 1024 * 1024; // 1 MiB #if __APPLE__ && __clang__ static __thread Int64 local_delta{}; #else diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index c87ec713dda..3f4122edf9f 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -19,7 +19,7 @@ #include - +extern std::atomic real_rss; namespace CurrentMetrics { extern const Metric MemoryTracking; @@ -35,6 +35,9 @@ class MemoryTracker std::atomic peak{0}; std::atomic limit{0}; + // How many bytes RSS(Resident Set Size) can be larger than limit(max_memory_usage_for_all_queries). Default: 5GB + Int64 bytes_rss_larger_than_limit = 5368709120; + /// To test exception safety of calling code, memory tracker throws an exception on each memory allocation with specified probability. double fault_probability = 0; @@ -70,6 +73,8 @@ class MemoryTracker Int64 getPeak() const { return peak.load(std::memory_order_relaxed); } + Int64 getLimit() const { return limit.load(std::memory_order_relaxed); } + void setLimit(Int64 limit_) { limit.store(limit_, std::memory_order_relaxed); } /** Set limit if it was not set. @@ -77,6 +82,8 @@ class MemoryTracker */ void setOrRaiseLimit(Int64 value); + void setBytesThatRssLargerThanLimit(Int64 value) { bytes_rss_larger_than_limit = value; } + void setFaultProbability(double value) { fault_probability = value; } /// next should be changed only once: from nullptr to some value. diff --git a/dbms/src/Common/tests/gtest_memtracker.cpp b/dbms/src/Common/tests/gtest_memtracker.cpp new file mode 100644 index 00000000000..d31e7b42df4 --- /dev/null +++ b/dbms/src/Common/tests/gtest_memtracker.cpp @@ -0,0 +1,121 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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::tests +{ +namespace +{ +class MemTrackerTest : public ::testing::Test +{ +}; + +TEST_F(MemTrackerTest, testBasic) +try +{ + MemoryTracker mem_tracker; + mem_tracker.alloc(1024); + ASSERT_EQ(1024, mem_tracker.get()); + mem_tracker.free(1024); + ASSERT_EQ(0, mem_tracker.get()); +} +CATCH + +TEST_F(MemTrackerTest, testRootAndChild) +try +{ + MemoryTracker root_mem_tracker; + MemoryTracker child_mem_tracker(512); + child_mem_tracker.setNext(&root_mem_tracker); + // alloc 500 + child_mem_tracker.alloc(500); + ASSERT_EQ(500, child_mem_tracker.get()); + ASSERT_EQ(500, root_mem_tracker.get()); + + // alloc 256 base on 500 + bool has_err = false; + try + { + child_mem_tracker.alloc(256); //500 + 256 > limit(512) + } + catch (...) + { + has_err = true; + } + ASSERT_TRUE(has_err); + ASSERT_EQ(500, child_mem_tracker.get()); + ASSERT_EQ(500, root_mem_tracker.get()); + + //free 500 + child_mem_tracker.free(500); + ASSERT_EQ(0, child_mem_tracker.get()); + ASSERT_EQ(0, root_mem_tracker.get()); +} +CATCH + +TEST_F(MemTrackerTest, testRootAndMultipleChild) +try +{ + MemoryTracker root(512); // limit 512 + MemoryTracker child1(512); // limit 512 + MemoryTracker child2(512); // limit 512 + child1.setNext(&root); + child2.setNext(&root); + // alloc 500 on child1 + child1.alloc(500); + ASSERT_EQ(500, child1.get()); + ASSERT_EQ(0, child2.get()); + ASSERT_EQ(500, root.get()); + + + // alloc 500 on child2, should fail + bool has_err = false; + try + { + child2.alloc(500); // root will throw error because of "out of quota" + } + catch (...) + { + has_err = true; + } + ASSERT_TRUE(has_err); + ASSERT_EQ(500, child1.get()); + ASSERT_EQ(0, child2.get()); + ASSERT_EQ(500, root.get()); + + // alloc 10 on child2 + child2.alloc(10); + ASSERT_EQ(500, child1.get()); + ASSERT_EQ(10, child2.get()); + ASSERT_EQ(510, root.get()); + + // free 500 on child1 + child1.free(500); + ASSERT_EQ(0, child1.get()); + ASSERT_EQ(10, child2.get()); + ASSERT_EQ(10, root.get()); + + // free 10 on child2 + child2.free(10); + ASSERT_EQ(0, child1.get()); + ASSERT_EQ(0, child2.get()); + ASSERT_EQ(0, root.get()); +} +CATCH + + +} // namespace +} // namespace DB::tests diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index df993d8e6e9..3654a437bf7 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -120,8 +120,6 @@ DBGInvoker::DBGInvoker() regSchemafulFunc("query_mapped", dbgFuncQueryMapped); regSchemalessFunc("get_tiflash_replica_count", dbgFuncGetTiflashReplicaCount); regSchemalessFunc("get_partition_tables_tiflash_replica_count", dbgFuncGetPartitionTablesTiflashReplicaCount); - regSchemalessFunc("get_tiflash_mode", dbgFuncGetTiflashMode); - regSchemalessFunc("get_partition_tables_tiflash_mode", dbgFuncGetPartitionTablesTiflashMode); regSchemalessFunc("search_log_for_key", dbgFuncSearchLogForKey); regSchemalessFunc("tidb_dag", dbgFuncTiDBQueryFromNaturalDag); diff --git a/dbms/src/Debug/dbgFuncMisc.cpp b/dbms/src/Debug/dbgFuncMisc.cpp index a1d592f88e9..b6f4521b4fa 100644 --- a/dbms/src/Debug/dbgFuncMisc.cpp +++ b/dbms/src/Debug/dbgFuncMisc.cpp @@ -25,13 +25,24 @@ namespace DB { inline size_t getReadTSOForLog(const String & line) { - auto sub_line = line.substr(line.find("read_tso=")); - std::regex rx(R"((0|[1-9][0-9]*))"); - std::smatch m; - if (regex_search(sub_line, m, rx)) - return std::stoul(m[1]); - else - return 0; + try + { + std::regex rx(R"((0|[1-9][0-9]*))"); + std::smatch m; + auto pos = line.find("read_tso="); + if (pos != std::string::npos && regex_search(line.cbegin() + pos, line.cend(), m, rx)) + { + return std::stoul(m[1]); + } + else + { + return 0; + } + } + catch (std::exception & e) + { + throw Exception(fmt::format("Parse 'read tso' failed, exception: {}, line {}", e.what(), line)); + } } // Usage example: @@ -90,13 +101,24 @@ void dbgFuncSearchLogForKey(Context & context, const ASTs & args, DBGInvoker::Pr } } // try parse the first number following the key - auto sub_line = target_line.substr(target_line.find(key)); - std::regex rx(R"([+-]?([0-9]+([.][0-9]*)?|[.][0-9]+))"); - std::smatch m; - if (regex_search(sub_line, m, rx)) - output(m[1]); - else - output("Invalid"); + try + { + std::regex rx(R"([+-]?([0-9]+([.][0-9]*)?|[.][0-9]+))"); + std::smatch m; + auto pos = target_line.find(key); + if (pos != std::string::npos && regex_search(target_line.cbegin() + pos, target_line.cend(), m, rx)) + { + output(m[1]); + } + else + { + output("Invalid"); + } + } + catch (std::exception & e) + { + throw Exception(fmt::format("Parse 'RSFilter exclude rate' failed, exception: {}, target_line {}", e.what(), target_line)); + } } void dbgFuncTriggerGlobalPageStorageGC(Context & context, const ASTs & /*args*/, DBGInvoker::Printer /*output*/) diff --git a/dbms/src/Debug/dbgFuncSchemaName.cpp b/dbms/src/Debug/dbgFuncSchemaName.cpp index 3aa7b6e3af4..c9bd958805f 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.cpp +++ b/dbms/src/Debug/dbgFuncSchemaName.cpp @@ -180,57 +180,4 @@ void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs output(fmt_buf.toString()); } - -void dbgFuncGetTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output) -{ - if (args.empty() || args.size() != 2) - throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); - - const String & database_name = typeid_cast(*args[0]).name; - FmtBuffer fmt_buf; - - const String & table_name = typeid_cast(*args[1]).name; - auto mapped = mappedTable(context, database_name, table_name); - auto storage = context.getTable(mapped->first, mapped->second); - auto managed_storage = std::dynamic_pointer_cast(storage); - if (!managed_storage) - throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); - - fmt_buf.append((TiFlashModeToString(managed_storage->getTableInfo().tiflash_mode))); - - output(fmt_buf.toString()); -} - -void dbgFuncGetPartitionTablesTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output) -{ - if (args.empty() || args.size() != 2) - throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); - - const String & database_name = typeid_cast(*args[0]).name; - FmtBuffer fmt_buf; - - const String & table_name = typeid_cast(*args[1]).name; - auto mapped = mappedTable(context, database_name, table_name); - auto storage = context.getTable(mapped->first, mapped->second); - auto managed_storage = std::dynamic_pointer_cast(storage); - if (!managed_storage) - throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); - - auto table_info = managed_storage->getTableInfo(); - - if (!table_info.isLogicalPartitionTable()) - throw Exception(database_name + "." + table_name + " is not logical partition table", ErrorCodes::BAD_ARGUMENTS); - - SchemaNameMapper name_mapper; - for (const auto & part_def : table_info.partition.definitions) - { - auto paritition_table_info = table_info.producePartitionTableInfo(part_def.id, name_mapper); - auto partition_storage = context.getTMTContext().getStorages().get(paritition_table_info->id); - fmt_buf.append((TiFlashModeToString(partition_storage->getTableInfo().tiflash_mode))); - fmt_buf.append("/"); - } - - output(fmt_buf.toString()); -} - } // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchemaName.h b/dbms/src/Debug/dbgFuncSchemaName.h index ec18f89e911..2a31b3a7f6d 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.h +++ b/dbms/src/Debug/dbgFuncSchemaName.h @@ -50,14 +50,4 @@ void dbgFuncGetTiflashReplicaCount(Context & context, const ASTs & args, DBGInvo // ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_replica_count(db_name, table_name)" void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output); -// Get table's tiflash mode with mapped table name -// Usage: -// ./storage-client.sh "DBGInvoke get_tiflash_mode(db_name, table_name)" -void dbgFuncGetTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output); - -// Get the logical table's partition tables' tiflash replica counts with mapped table name -// Usage: -// ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_mode(db_name, table_name)" -void dbgFuncGetPartitionTablesTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output); - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 6167090194a..46cb5e3bbf5 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -240,4 +240,8 @@ const SingleTableRegions & DAGContext::getTableRegionsInfoByTableID(Int64 table_ { return tables_regions_info.getTableRegionInfoByTableID(table_id); } +const MPPReceiverSetPtr & DAGContext::getMppReceiverSet() const +{ + return mpp_receiver_set; +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 93e7edda7e8..af5932ac444 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -305,6 +305,7 @@ class DAGContext { mpp_receiver_set = receiver_set; } + const MPPReceiverSetPtr & getMppReceiverSet() const; void addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader); std::vector & getCoprocessorReaders(); diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index fb4a03f0999..087949d2b89 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -600,6 +600,7 @@ std::unordered_map DAGStorageInterpreter::generateSele context.getTimezoneInfo()); query_info.req_id = fmt::format("{} Table<{}>", log->identifier(), table_id); query_info.keep_order = table_scan.keepOrder(); + query_info.is_fast_scan = table_scan.isFastScan(); return query_info; }; if (table_scan.isPartitionTableScan()) diff --git a/dbms/src/Flash/Coprocessor/StreamWriter.h b/dbms/src/Flash/Coprocessor/StreamWriter.h index fad403b0726..41b108492f5 100644 --- a/dbms/src/Flash/Coprocessor/StreamWriter.h +++ b/dbms/src/Flash/Coprocessor/StreamWriter.h @@ -54,7 +54,7 @@ struct StreamWriter { ::coprocessor::BatchResponse resp; if (!response.SerializeToString(resp.mutable_data())) - throw Exception("Fail to serialize response, response size: " + std::to_string(response.ByteSizeLong())); + throw Exception("[StreamWriter]Fail to serialize response, response size: " + std::to_string(response.ByteSizeLong())); std::lock_guard lk(write_mutex); if (!writer->Write(resp)) throw Exception("Failed to write resp"); diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp index 6e70f280e6f..5fa7eb245ab 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -135,62 +136,61 @@ template template void StreamingDAGResponseWriter::encodeThenWriteBlocks( const std::vector & input_blocks, - tipb::SelectResponse & response) const + TrackedSelectResp & response) const { if (dag_context.encode_type == tipb::EncodeType::TypeCHBlock) { if (dag_context.isMPPTask()) /// broadcast data among TiFlash nodes in MPP { - mpp::MPPDataPacket packet; + TrackedMppDataPacket tracked_packet(current_memory_tracker); if constexpr (send_exec_summary_at_last) { - serializeToPacket(packet, response); + tracked_packet.serializeByResponse(response.getResponse()); } if (input_blocks.empty()) { if constexpr (send_exec_summary_at_last) { - writer->write(packet); + writer->write(tracked_packet.getPacket()); } return; } for (const auto & block : input_blocks) { chunk_codec_stream->encode(block, 0, block.rows()); - packet.add_chunks(chunk_codec_stream->getString()); + tracked_packet.addChunk(chunk_codec_stream->getString()); chunk_codec_stream->clear(); } - writer->write(packet); + writer->write(tracked_packet.getPacket()); } else /// passthrough data to a non-TiFlash node, like sending data to TiSpark { - response.set_encode_type(dag_context.encode_type); + response.setEncodeType(dag_context.encode_type); if (input_blocks.empty()) { if constexpr (send_exec_summary_at_last) { - writer->write(response); + writer->write(response.getResponse()); } return; } for (const auto & block : input_blocks) { chunk_codec_stream->encode(block, 0, block.rows()); - auto * dag_chunk = response.add_chunks(); - dag_chunk->set_rows_data(chunk_codec_stream->getString()); + response.addChunk(chunk_codec_stream->getString()); chunk_codec_stream->clear(); } - writer->write(response); + writer->write(response.getResponse()); } } else /// passthrough data to a TiDB node { - response.set_encode_type(dag_context.encode_type); + response.setEncodeType(dag_context.encode_type); if (input_blocks.empty()) { if constexpr (send_exec_summary_at_last) { - writer->write(response); + writer->write(response.getResponse()); } return; } @@ -203,8 +203,7 @@ void StreamingDAGResponseWriter::e { if (current_records_num >= records_per_chunk) { - auto * dag_chunk = response.add_chunks(); - dag_chunk->set_rows_data(chunk_codec_stream->getString()); + response.addChunk(chunk_codec_stream->getString()); chunk_codec_stream->clear(); current_records_num = 0; } @@ -217,11 +216,10 @@ void StreamingDAGResponseWriter::e if (current_records_num > 0) { - auto * dag_chunk = response.add_chunks(); - dag_chunk->set_rows_data(chunk_codec_stream->getString()); + response.addChunk(chunk_codec_stream->getString()); chunk_codec_stream->clear(); } - writer->write(response); + writer->write(response.getResponse()); } } @@ -230,9 +228,9 @@ template template void StreamingDAGResponseWriter::batchWrite() { - tipb::SelectResponse response; + TrackedSelectResp response; if constexpr (send_exec_summary_at_last) - addExecuteSummaries(response, !dag_context.isMPPTask() || dag_context.isRootMPPTask()); + addExecuteSummaries(response.getResponse(), !dag_context.isMPPTask() || dag_context.isRootMPPTask()); if (exchange_type == tipb::ExchangeType::Hash) { partitionAndEncodeThenWriteBlocks(blocks, response); @@ -249,13 +247,13 @@ template template void StreamingDAGResponseWriter::handleExecSummary( const std::vector & input_blocks, - std::vector & packet, + std::vector & packets, tipb::SelectResponse & response) const { if constexpr (send_exec_summary_at_last) { /// Sending the response to only one node, default the first one. - serializeToPacket(packet[0], response); + packets[0].serializeByResponse(response); // No need to send data when blocks are not empty, // because exec_summary will be sent together with blocks. @@ -263,7 +261,7 @@ void StreamingDAGResponseWriter::h { for (auto part_id = 0; part_id < partition_num; ++part_id) { - writer->write(packet[part_id], part_id); + writer->write(packets[part_id].getPacket(), part_id); } } } @@ -273,18 +271,18 @@ template template void StreamingDAGResponseWriter::writePackets( const std::vector & responses_row_count, - std::vector & packets) const + std::vector & packets) const { for (size_t part_id = 0; part_id < packets.size(); ++part_id) { if constexpr (send_exec_summary_at_last) { - writer->write(packets[part_id], part_id); + writer->write(packets[part_id].getPacket(), part_id); } else { if (responses_row_count[part_id] > 0) - writer->write(packets[part_id], part_id); + writer->write(packets[part_id].getPacket(), part_id); } } } @@ -354,12 +352,12 @@ template template void StreamingDAGResponseWriter::partitionAndEncodeThenWriteBlocks( std::vector & input_blocks, - tipb::SelectResponse & response) const + TrackedSelectResp & response) const { static_assert(!enable_fine_grained_shuffle); - std::vector packet(partition_num); + std::vector tracked_packets(partition_num); std::vector responses_row_count(partition_num); - handleExecSummary(input_blocks, packet, response); + handleExecSummary(input_blocks, tracked_packets, response.getResponse()); if (input_blocks.empty()) return; @@ -378,12 +376,12 @@ void StreamingDAGResponseWriter::p dest_block.setColumns(std::move(dest_tbl_cols[part_id])); responses_row_count[part_id] += dest_block.rows(); chunk_codec_stream->encode(dest_block, 0, dest_block.rows()); - packet[part_id].add_chunks(chunk_codec_stream->getString()); + tracked_packets[part_id].addChunk(chunk_codec_stream->getString()); chunk_codec_stream->clear(); } } - writePackets(responses_row_count, packet); + writePackets(responses_row_count, tracked_packets); } /// Hash exchanging data among only TiFlash nodes. Only be called when enable_fine_grained_shuffle is true. @@ -399,12 +397,12 @@ void StreamingDAGResponseWriter::b if constexpr (send_exec_summary_at_last) addExecuteSummaries(response, !dag_context.isMPPTask() || dag_context.isRootMPPTask()); - std::vector packet(partition_num); + std::vector tracked_packets(partition_num); std::vector responses_row_count(partition_num, 0); // fine_grained_shuffle_stream_count is in [0, 1024], and partition_num is uint16_t, so will not overflow. uint32_t bucket_num = partition_num * fine_grained_shuffle_stream_count; - handleExecSummary(blocks, packet, response); + handleExecSummary(blocks, tracked_packets, response); if (!blocks.empty()) { std::vector final_dest_tbl_columns(bucket_num); @@ -441,15 +439,15 @@ void StreamingDAGResponseWriter::b row_count_per_part += dest_block.rows(); chunk_codec_stream->encode(dest_block, 0, dest_block.rows()); - packet[part_id].add_chunks(chunk_codec_stream->getString()); - packet[part_id].add_stream_ids(stream_idx); + tracked_packets[part_id].addChunk(chunk_codec_stream->getString()); + tracked_packets[part_id].packet.add_stream_ids(stream_idx); chunk_codec_stream->clear(); } responses_row_count[part_id] = row_count_per_part; } } - writePackets(responses_row_count, packet); + writePackets(responses_row_count, tracked_packets); blocks.clear(); rows_in_blocks = 0; diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h index cd7559d1e79..1e37090509b 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h @@ -23,6 +23,7 @@ #include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" +#include #include #include @@ -58,16 +59,16 @@ class StreamingDAGResponseWriter : public DAGResponseWriter void batchWriteFineGrainedShuffle(); template - void encodeThenWriteBlocks(const std::vector & input_blocks, tipb::SelectResponse & response) const; + void encodeThenWriteBlocks(const std::vector & input_blocks, TrackedSelectResp & response) const; template - void partitionAndEncodeThenWriteBlocks(std::vector & input_blocks, tipb::SelectResponse & response) const; + void partitionAndEncodeThenWriteBlocks(std::vector & input_blocks, TrackedSelectResp & response) const; template void handleExecSummary(const std::vector & input_blocks, - std::vector & packet, + std::vector & packet, tipb::SelectResponse & response) const; template - void writePackets(const std::vector & responses_row_count, std::vector & packets) const; + void writePackets(const std::vector & responses_row_count, std::vector & packets) const; Int64 batch_send_min_limit; bool should_send_exec_summary_at_last; /// only one stream needs to sending execution summaries at last. diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp index dca738a2e6c..20a12770c73 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -27,6 +27,7 @@ TiDBTableScan::TiDBTableScan( // Only No-partition table need keep order when tablescan executor required keep order. // If keep_order is not set, keep order for safety. , keep_order(!is_partition_table_scan && (table_scan->tbl_scan().keep_order() || !table_scan->tbl_scan().has_keep_order())) + , is_fast_scan(table_scan->tbl_scan().is_fast_scan()) { if (is_partition_table_scan) { @@ -73,6 +74,7 @@ void TiDBTableScan::constructTableScanForRemoteRead(tipb::TableScan * tipb_table tipb_table_scan->set_next_read_engine(tipb::EngineType::Local); for (auto id : partition_table_scan.primary_prefix_column_ids()) tipb_table_scan->add_primary_prefix_column_ids(id); + tipb_table_scan->set_is_fast_scan(partition_table_scan.is_fast_scan()); } else { diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.h b/dbms/src/Flash/Coprocessor/TiDBTableScan.h index a5a463a8ff2..574d4b0a15f 100644 --- a/dbms/src/Flash/Coprocessor/TiDBTableScan.h +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -56,6 +56,11 @@ class TiDBTableScan return keep_order; } + bool isFastScan() const + { + return is_fast_scan; + } + private: const tipb::Executor * table_scan; String executor_id; @@ -71,6 +76,7 @@ class TiDBTableScan std::vector physical_table_ids; Int64 logical_table_id; bool keep_order; + bool is_fast_scan; }; } // namespace DB diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index c896757c84a..3cdb0c2a184 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -59,26 +59,27 @@ String getReceiverStateStr(const ExchangeReceiverState & s) template bool pushPacket(size_t source_index, const String & req_info, - MPPDataPacketPtr & packet, + const TrackedMppDataPacketPtr & tracked_packet, const std::vector & msg_channels, LoggerPtr & log) { bool push_succeed = true; const mpp::Error * error_ptr = nullptr; - if (packet->has_error()) - error_ptr = &packet->error(); + auto & packet = tracked_packet->packet; + if (packet.has_error()) + error_ptr = &packet.error(); const String * resp_ptr = nullptr; - if (!packet->data().empty()) - resp_ptr = &packet->data(); + if (!packet.data().empty()) + resp_ptr = &packet.data(); if constexpr (enable_fine_grained_shuffle) { std::vector> chunks(msg_channels.size()); - if (!packet->chunks().empty()) + if (!packet.chunks().empty()) { // Packet not empty. - if (unlikely(packet->stream_ids().empty())) + if (unlikely(packet.stream_ids().empty())) { // Fine grained shuffle is enabled in receiver, but sender didn't. We cannot handle this, so return error. // This can happen when there are old version nodes when upgrading. @@ -90,12 +91,12 @@ bool pushPacket(size_t source_index, } // packet.stream_ids[i] is corresponding to packet.chunks[i], // indicating which stream_id this chunk belongs to. - assert(packet->chunks_size() == packet->stream_ids_size()); + assert(packet.chunks_size() == packet.stream_ids_size()); - for (int i = 0; i < packet->stream_ids_size(); ++i) + for (int i = 0; i < packet.stream_ids_size(); ++i) { - UInt64 stream_id = packet->stream_ids(i) % msg_channels.size(); - chunks[stream_id].push_back(&packet->chunks(i)); + UInt64 stream_id = packet.stream_ids(i) % msg_channels.size(); + chunks[stream_id].push_back(&packet.chunks(i)); } } // Still need to send error_ptr or resp_ptr even if packet.chunks_size() is zero. @@ -107,7 +108,7 @@ bool pushPacket(size_t source_index, std::shared_ptr recv_msg = std::make_shared( source_index, req_info, - packet, + tracked_packet, error_ptr, resp_ptr, std::move(chunks[i])); @@ -123,10 +124,10 @@ bool pushPacket(size_t source_index, } else { - std::vector chunks(packet->chunks_size()); - for (int i = 0; i < packet->chunks_size(); ++i) + std::vector chunks(packet.chunks_size()); + for (int i = 0; i < packet.chunks_size(); ++i) { - chunks[i] = &packet->chunks(i); + chunks[i] = &packet.chunks(i); } if (!(resp_ptr == nullptr && error_ptr == nullptr && chunks.empty())) @@ -134,7 +135,7 @@ bool pushPacket(size_t source_index, std::shared_ptr recv_msg = std::make_shared( source_index, req_info, - packet, + tracked_packet, error_ptr, resp_ptr, std::move(chunks)); @@ -190,7 +191,7 @@ class AsyncRequestHandler : public UnaryCallback { packets.resize(batch_packet_count); for (auto & packet : packets) - packet = std::make_shared(); + packet = std::make_shared(); start(); } @@ -212,7 +213,7 @@ class AsyncRequestHandler : public UnaryCallback case AsyncRequestStage::WAIT_BATCH_READ: if (ok) ++read_packet_index; - if (!ok || read_packet_index == batch_packet_count || packets[read_packet_index - 1]->has_error()) + if (!ok || read_packet_index == batch_packet_count || packets[read_packet_index - 1]->hasError()) notifyReactor(); else reader->read(packets[read_packet_index], thisAsUnaryCallback()); @@ -228,6 +229,7 @@ class AsyncRequestHandler : public UnaryCallback // handle will be called by ExchangeReceiver::reactor. void handle() { + std::string err_info; LOG_FMT_TRACE(log, "stage: {}", stage); switch (stage) { @@ -251,8 +253,8 @@ class AsyncRequestHandler : public UnaryCallback if (auto packet = getErrorPacket()) setDone("Exchange receiver meet error : " + packet->error().msg()); - else if (!sendPackets()) - setDone("Exchange receiver meet error : push packets fail"); + else if (!sendPackets(err_info)) + setDone("Exchange receiver meet error : push packets fail, " + err_info); else if (read_packet_index < batch_packet_count) { stage = AsyncRequestStage::WAIT_FINISH; @@ -314,10 +316,10 @@ class AsyncRequestHandler : public UnaryCallback notify_queue->push(this); } - MPPDataPacketPtr getErrorPacket() const + TrackedMppDataPacketPtr getErrorPacket() const { // only the last packet may has error, see execute(). - if (read_packet_index != 0 && packets[read_packet_index - 1]->has_error()) + if (read_packet_index != 0 && packets[read_packet_index - 1]->hasError()) return packets[read_packet_index - 1]; return nullptr; } @@ -357,15 +359,31 @@ class AsyncRequestHandler : public UnaryCallback setDone(done_msg); } - bool sendPackets() + bool sendPackets(std::string & err_info) { + // note: no exception should be thrown rudely, since it's called by a GRPC poller. for (size_t i = 0; i < read_packet_index; ++i) { auto & packet = packets[i]; - if (!pushPacket(request->source_index, req_info, packet, *msg_channels, log)) + // We shouldn't throw error directly, since the caller works in a standalone thread. + try + { + packet->recomputeTrackedMem(); + if (!pushPacket( + request->source_index, + req_info, + packet, + *msg_channels, + log)) + return false; + } + catch (...) + { + err_info = getCurrentExceptionMessage(false); return false; + } // can't reuse packet since it is sent to readers. - packet = std::make_shared(); + packet = std::make_shared(); } return true; } @@ -390,7 +408,7 @@ class AsyncRequestHandler : public UnaryCallback AsyncRequestStage stage = AsyncRequestStage::NEED_INIT; std::shared_ptr reader; - MPPDataPacketPtrs packets; + TrackedMPPDataPacketPtrs packets; size_t read_packet_index = 0; Status finish_status = RPCContext::getStatusOK(); LoggerPtr log; @@ -405,7 +423,8 @@ ExchangeReceiverBase::ExchangeReceiverBase( size_t max_streams_, const String & req_id, const String & executor_id, - uint64_t fine_grained_shuffle_stream_count_) + uint64_t fine_grained_shuffle_stream_count_, + bool setup_conn_manually) : rpc_context(std::move(rpc_context_)) , source_num(source_num_) , max_streams(max_streams_) @@ -431,7 +450,11 @@ ExchangeReceiverBase::ExchangeReceiverBase( msg_channels.push_back(std::make_unique>>(max_buffer_size)); } rpc_context->fillSchema(schema); - setUpConnection(); + if (!setup_conn_manually) + { + // In CH client case, we need setUpConn right now. However, MPPTask will setUpConnection manually after ProcEntry is created. + setUpConnection(); + } } catch (...) { @@ -479,6 +502,8 @@ void ExchangeReceiverBase::close() template void ExchangeReceiverBase::setUpConnection() { + if (thread_count) + return; std::vector async_requests; for (size_t index = 0; index < source_num; ++index) @@ -601,15 +626,20 @@ void ExchangeReceiverBase::readLoop(const Request & req) for (;;) { LOG_FMT_TRACE(log, "begin next "); - MPPDataPacketPtr packet = std::make_shared(); + TrackedMppDataPacketPtr packet = std::make_shared(); bool success = reader->read(packet); if (!success) break; has_data = true; - if (packet->has_error()) + if (packet->hasError()) throw Exception("Exchange receiver meet error : " + packet->error().msg()); - if (!pushPacket(req.source_index, req_info, packet, msg_channels, log)) + if (!pushPacket( + req.source_index, + req_info, + packet, + msg_channels, + log)) { meet_error = true; auto local_state = getState(); @@ -681,9 +711,10 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( if (recv_msg->chunks.empty()) return detail; + auto & packet = recv_msg->packet->packet; // Record total packet size even if fine grained shuffle is enabled. - detail.packet_bytes = recv_msg->packet->ByteSizeLong(); + detail.packet_bytes = packet.ByteSizeLong(); for (const String * chunk : recv_msg->chunks) { diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index 9213eb76e60..8eed7878545 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -38,7 +38,7 @@ struct ReceivedMessage size_t source_index; String req_info; // shared_ptr is copied to make sure error_ptr, resp_ptr and chunks are valid. - const std::shared_ptr packet; + const std::shared_ptr packet; const mpp::Error * error_ptr; const String * resp_ptr; std::vector chunks; @@ -46,7 +46,7 @@ struct ReceivedMessage // Constructor that move chunks. ReceivedMessage(size_t source_index_, const String & req_info_, - const std::shared_ptr & packet_, + const std::shared_ptr & packet_, const mpp::Error * error_ptr_, const String * resp_ptr_, std::vector && chunks_) @@ -129,10 +129,13 @@ class ExchangeReceiverBase size_t max_streams_, const String & req_id, const String & executor_id, - uint64_t fine_grained_shuffle_stream_count); + uint64_t fine_grained_shuffle_stream_count, + bool setup_conn_manually = false); ~ExchangeReceiverBase(); + void setUpConnection(); + void cancel(); void close(); @@ -166,7 +169,6 @@ class ExchangeReceiverBase private: using Request = typename RPCContext::Request; - void setUpConnection(); // Template argument enable_fine_grained_shuffle will be setup properly in setUpConnection(). template void readLoop(const Request & req); diff --git a/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp b/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp index 310745aa024..236d15f9093 100644 --- a/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp +++ b/dbms/src/Flash/Mpp/GRPCReceiverContext.cpp @@ -63,9 +63,9 @@ struct GrpcExchangePacketReader : public ExchangePacketReader call = std::make_shared>(req.req); } - bool read(MPPDataPacketPtr & packet) override + bool read(TrackedMppDataPacketPtr & packet) override { - return reader->Read(packet.get()); + return packet->read(reader); } ::grpc::Status finish() override @@ -101,9 +101,9 @@ struct AsyncGrpcExchangePacketReader : public AsyncExchangePacketReader callback); } - void read(MPPDataPacketPtr & packet, UnaryCallback * callback) override + void read(TrackedMppDataPacketPtr & packet, UnaryCallback * callback) override { - reader->Read(packet.get(), callback); + packet->read(reader, callback); } void finish(::grpc::Status & status, UnaryCallback * callback) override @@ -131,9 +131,9 @@ struct LocalExchangePacketReader : public ExchangePacketReader } } - bool read(MPPDataPacketPtr & packet) override + bool read(TrackedMppDataPacketPtr & packet) override { - MPPDataPacketPtr tmp_packet = local_tunnel_sender->readForLocal(); + TrackedMppDataPacketPtr tmp_packet = local_tunnel_sender->readForLocal(); bool success = tmp_packet != nullptr; if (success) packet = tmp_packet; diff --git a/dbms/src/Flash/Mpp/GRPCReceiverContext.h b/dbms/src/Flash/Mpp/GRPCReceiverContext.h index 3868271ff8a..a6e9afb059b 100644 --- a/dbms/src/Flash/Mpp/GRPCReceiverContext.h +++ b/dbms/src/Flash/Mpp/GRPCReceiverContext.h @@ -28,14 +28,14 @@ namespace DB { using MPPDataPacket = mpp::MPPDataPacket; -using MPPDataPacketPtr = std::shared_ptr; -using MPPDataPacketPtrs = std::vector; +using TrackedMppDataPacketPtr = std::shared_ptr; +using TrackedMPPDataPacketPtrs = std::vector; class ExchangePacketReader { public: virtual ~ExchangePacketReader() = default; - virtual bool read(MPPDataPacketPtr & packet) = 0; + virtual bool read(TrackedMppDataPacketPtr & packet) = 0; virtual ::grpc::Status finish() = 0; }; using ExchangePacketReaderPtr = std::shared_ptr; @@ -45,7 +45,7 @@ class AsyncExchangePacketReader public: virtual ~AsyncExchangePacketReader() = default; virtual void init(UnaryCallback * callback) = 0; - virtual void read(MPPDataPacketPtr & packet, UnaryCallback * callback) = 0; + virtual void read(TrackedMppDataPacketPtr & packet, UnaryCallback * callback) = 0; virtual void finish(::grpc::Status & status, UnaryCallback * callback) = 0; }; using AsyncExchangePacketReaderPtr = std::shared_ptr; diff --git a/dbms/src/Flash/Mpp/MPPReceiverSet.cpp b/dbms/src/Flash/Mpp/MPPReceiverSet.cpp index fd8da091224..c433d8aa9a9 100644 --- a/dbms/src/Flash/Mpp/MPPReceiverSet.cpp +++ b/dbms/src/Flash/Mpp/MPPReceiverSet.cpp @@ -44,6 +44,15 @@ void MPPReceiverSet::cancel() cop_reader->cancel(); } + +void MPPReceiverSet::setUpConnection() +{ + for (auto & it : exchange_receiver_map) + { + it.second->setUpConnection(); + } +} + void MPPReceiverSet::close() { for (auto & it : exchange_receiver_map) diff --git a/dbms/src/Flash/Mpp/MPPReceiverSet.h b/dbms/src/Flash/Mpp/MPPReceiverSet.h index 367fd6859ce..71ae8606e8f 100644 --- a/dbms/src/Flash/Mpp/MPPReceiverSet.h +++ b/dbms/src/Flash/Mpp/MPPReceiverSet.h @@ -28,6 +28,7 @@ class MPPReceiverSet void addExchangeReceiver(const String & executor_id, const ExchangeReceiverPtr & exchange_receiver); void addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader); ExchangeReceiverPtr getExchangeReceiver(const String & executor_id) const; + void setUpConnection(); void cancel(); void close(); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 1444d2f1963..121352595aa 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -201,7 +201,8 @@ void MPPTask::initExchangeReceivers() context->getMaxStreams(), log->identifier(), executor_id, - executor.fine_grained_shuffle_stream_count()); + executor.fine_grained_shuffle_stream_count(), + true); if (status != RUNNING) throw Exception("exchange receiver map can not be initialized, because the task is not in running state"); diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index b1a25095ba0..7a9f4cdea76 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -68,8 +68,9 @@ MPPTunnel::MPPTunnel( , status(TunnelStatus::Unconnected) , timeout(timeout_) , tunnel_id(tunnel_id_) - , send_queue(std::make_shared>(std::max(5, input_steams_num_ * 5))) // MPMCQueue can benefit from a slightly larger queue size + , send_queue(std::make_shared>(std::max(5, input_steams_num_ * 5))) // MPMCQueue can benefit from a slightly larger queue size , log(Logger::get("MPPTunnel", req_id, tunnel_id)) + , mem_tracker(current_memory_tracker) { RUNTIME_ASSERT(!(is_local_ && is_async_), log, "is_local: {}, is_async: {}.", is_local_, is_async_); if (is_local_) @@ -109,6 +110,10 @@ void MPPTunnel::finishSendQueue() /// exit abnormally, such as being cancelled. void MPPTunnel::close(const String & reason) { + SCOPE_EXIT({ + // ensure the tracked memory is released and udpated before memotry tracker(in ProcListEntry) is released + send_queue->finishAndDrain(); // drain the send_queue when close + }); { std::unique_lock lk(*mu); switch (status) @@ -124,7 +129,7 @@ void MPPTunnel::close(const String & reason) try { FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_close_tunnel); - send_queue->push(std::make_shared(getPacketWithError(reason))); + send_queue->push(std::make_shared(getPacketWithError(reason), mem_tracker)); if (mode == TunnelSenderMode::ASYNC_GRPC) async_tunnel_sender->tryFlushOne(); } @@ -159,7 +164,7 @@ void MPPTunnel::write(const mpp::MPPDataPacket & data, bool close_after_write) if (status == TunnelStatus::Finished) throw Exception(fmt::format("write to tunnel which is already closed,{}", tunnel_sender ? tunnel_sender->getConsumerFinishMsg() : "")); - if (send_queue->push(std::make_shared(data)) == MPMCQueueResult::OK) + if (send_queue->push(std::make_shared(data, mem_tracker)) == MPMCQueueResult::OK) { connection_profile_info.bytes += data.ByteSizeLong(); connection_profile_info.packets += 1; @@ -301,6 +306,11 @@ StringRef MPPTunnel::statusToString() } } +void MPPTunnel::updateMemTracker() +{ + mem_tracker = current_memory_tracker; +} + void TunnelSender::consumerFinish(const String & msg) { LOG_FMT_TRACE(log, "calling consumer Finish"); @@ -321,10 +331,10 @@ void SyncTunnelSender::sendJob() String err_msg; try { - MPPDataPacketPtr res; + TrackedMppDataPacketPtr res; while (send_queue->pop(res) == MPMCQueueResult::OK) { - if (!writer->write(*res)) + if (!writer->write(res->packet)) { err_msg = "grpc writes failed."; break; @@ -379,11 +389,11 @@ void AsyncTunnelSender::sendOne(bool use_lock) bool queue_empty_flag = false; try { - MPPDataPacketPtr res; + TrackedMppDataPacketPtr res; queue_empty_flag = send_queue->pop(res) != MPMCQueueResult::OK; if (!queue_empty_flag) { - if (!writer->write(*res)) + if (!writer->write(res->packet)) { err_msg = "grpc writes failed."; } @@ -413,11 +423,15 @@ void AsyncTunnelSender::sendOne(bool use_lock) } } -LocalTunnelSender::MPPDataPacketPtr LocalTunnelSender::readForLocal() +std::shared_ptr LocalTunnelSender::readForLocal() { - MPPDataPacketPtr res; + TrackedMppDataPacketPtr res; if (send_queue->pop(res) == MPMCQueueResult::OK) + { + // switch tunnel's memory tracker into receiver's + res->switchMemTracker(current_memory_tracker); return res; + } consumerFinish(""); return nullptr; } diff --git a/dbms/src/Flash/Mpp/MPPTunnel.h b/dbms/src/Flash/Mpp/MPPTunnel.h index 5243c9aaf36..e6e6ad94bbd 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.h +++ b/dbms/src/Flash/Mpp/MPPTunnel.h @@ -33,6 +33,8 @@ #include #pragma GCC diagnostic pop +#include + #include #include #include @@ -61,8 +63,8 @@ enum class TunnelSenderMode class TunnelSender : private boost::noncopyable { public: - using MPPDataPacketPtr = std::shared_ptr; - using DataPacketMPMCQueuePtr = std::shared_ptr>; + using TrackedMppDataPacketPtr = std::shared_ptr; + using DataPacketMPMCQueuePtr = std::shared_ptr>; virtual ~TunnelSender() = default; TunnelSender(TunnelSenderMode mode_, DataPacketMPMCQueuePtr send_queue_, PacketWriter * writer_, const LoggerPtr log_, const String & tunnel_id_) : mode(mode_) @@ -177,7 +179,7 @@ class LocalTunnelSender : public TunnelSender public: using Base = TunnelSender; using Base::Base; - MPPDataPacketPtr readForLocal(); + TrackedMppDataPacketPtr readForLocal(); }; using TunnelSenderPtr = std::shared_ptr; @@ -202,7 +204,7 @@ using LocalTunnelSenderPtr = std::shared_ptr; * To be short: before connect, only close can finish a MPPTunnel; after connect, only Sender Finish can. * * Each MPPTunnel has a Sender to consume data. There're three kinds of senders: sync_remote, local and async_remote. - * + * * The protocol between MPPTunnel and Sender: * - All data will be pushed into the `send_queue`, including errors. * - MPPTunnel may finish `send_queue` to notify Sender normally finish. @@ -259,6 +261,8 @@ class MPPTunnel : private boost::noncopyable const LoggerPtr & getLogger() const { return log; } + void updateMemTracker(); + TunnelSenderPtr getTunnelSender() { return tunnel_sender; } SyncTunnelSenderPtr getSyncTunnelSender() { return sync_tunnel_sender; } AsyncTunnelSenderPtr getAsyncTunnelSender() { return async_tunnel_sender; } @@ -292,11 +296,12 @@ class MPPTunnel : private boost::noncopyable // tunnel id is in the format like "tunnel[sender]+[receiver]" String tunnel_id; - using MPPDataPacketPtr = std::shared_ptr; - using DataPacketMPMCQueuePtr = std::shared_ptr>; + using TrackedMppDataPacketPtr = std::shared_ptr; + using DataPacketMPMCQueuePtr = std::shared_ptr>; DataPacketMPMCQueuePtr send_queue; ConnectionProfileInfo connection_profile_info; const LoggerPtr log; + MemoryTracker * mem_tracker; TunnelSenderMode mode; // Tunnel transfer data mode TunnelSenderPtr tunnel_sender; // Used to refer to one of sync/async/local_tunnel_sender which is not nullptr, just for coding convenience // According to mode value, among the sync/async/local_tunnel_senders, only the responding sender is not null and do actual work diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 3de5af31091..e6610d4d7b8 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include @@ -26,13 +27,6 @@ extern const char exception_during_mpp_write_err_to_tunnel[]; } // namespace FailPoints namespace { -inline mpp::MPPDataPacket serializeToPacket(const tipb::SelectResponse & response) -{ - mpp::MPPDataPacket packet; - if (!response.SerializeToString(packet.mutable_data())) - throw Exception(fmt::format("Fail to serialize response, response size: {}", response.ByteSizeLong())); - return packet; -} void checkPacketSize(size_t size) { @@ -57,11 +51,19 @@ void MPPTunnelSetBase::clearExecutionSummaries(tipb::SelectResponse & re } } +template +void MPPTunnelSetBase::updateMemTracker() +{ + for (size_t i = 0; i < tunnels.size(); ++i) + tunnels[i]->updateMemTracker(); +} + template void MPPTunnelSetBase::write(tipb::SelectResponse & response) { - auto packet = serializeToPacket(response); - tunnels[0]->write(packet); + TrackedMppDataPacket tracked_packet; + tracked_packet.serializeByResponse(response); + tunnels[0]->write(tracked_packet.getPacket()); if (tunnels.size() > 1) { @@ -69,10 +71,11 @@ void MPPTunnelSetBase::write(tipb::SelectResponse & response) if (response.execution_summaries_size() > 0) { clearExecutionSummaries(response); - packet = serializeToPacket(response); + tracked_packet = TrackedMppDataPacket(); + tracked_packet.serializeByResponse(response); } for (size_t i = 1; i < tunnels.size(); ++i) - tunnels[i]->write(packet); + tunnels[i]->write(tracked_packet.getPacket()); } } @@ -98,10 +101,11 @@ void MPPTunnelSetBase::write(mpp::MPPDataPacket & packet) template void MPPTunnelSetBase::write(tipb::SelectResponse & response, int16_t partition_id) { + TrackedMppDataPacket tracked_packet; if (partition_id != 0 && response.execution_summaries_size() > 0) clearExecutionSummaries(response); - - tunnels[partition_id]->write(serializeToPacket(response)); + tracked_packet.serializeByResponse(response); + tunnels[partition_id]->write(tracked_packet.getPacket()); } template diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index e4123db1be5..da37423876e 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -58,6 +58,7 @@ class MPPTunnelSetBase : private boost::noncopyable void close(const String & reason); void finishWrite(); void registerTunnel(const MPPTaskId & id, const TunnelPtr & tunnel); + void updateMemTracker(); TunnelPtr getTunnelByReceiverTaskId(const MPPTaskId & id); diff --git a/dbms/src/Flash/Mpp/TrackedMppDataPacket.h b/dbms/src/Flash/Mpp/TrackedMppDataPacket.h new file mode 100644 index 00000000000..7cb2103d4f3 --- /dev/null +++ b/dbms/src/Flash/Mpp/TrackedMppDataPacket.h @@ -0,0 +1,222 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#pragma GCC diagnostic ignored "-Wnon-virtual-dtor" +#ifdef __clang__ +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" +#endif +#include +#include +#include +#include +#include +#pragma GCC diagnostic pop +#include + +#include + +namespace DB +{ +inline size_t estimateAllocatedSize(const mpp::MPPDataPacket & data) +{ + size_t ret = data.data().size(); + for (int i = 0; i < data.chunks_size(); i++) + { + ret += data.chunks(i).size(); + } + return ret; +} + +struct MemTrackerWrapper +{ + MemTrackerWrapper(size_t _size, MemoryTracker * memory_tracker) + : memory_tracker(memory_tracker) + , size(0) + { + alloc(_size); + } + + explicit MemTrackerWrapper(MemoryTracker * memory_tracker) + : memory_tracker(memory_tracker) + , size(0) + {} + + void alloc(size_t delta) + { + if (delta) + { + if (memory_tracker) + { + memory_tracker->alloc(delta); + size += delta; + } + } + } + + void free(size_t delta) + { + if (delta) + { + if (memory_tracker) + { + memory_tracker->free(delta); + size -= delta; + } + } + } + + void switchMemTracker(MemoryTracker * new_memory_tracker) + { + int bak_size = size; + freeAll(); + memory_tracker = new_memory_tracker; + alloc(bak_size); + } + ~MemTrackerWrapper() + { + freeAll(); + } + + void freeAll() + { + free(size); + } + MemoryTracker * memory_tracker; + size_t size = 0; +}; + +struct TrackedMppDataPacket +{ + explicit TrackedMppDataPacket(const mpp::MPPDataPacket & data, MemoryTracker * memory_tracker) + : mem_tracker_wrapper(estimateAllocatedSize(data), memory_tracker) + { + packet = data; + } + + explicit TrackedMppDataPacket() + : mem_tracker_wrapper(current_memory_tracker) + {} + + explicit TrackedMppDataPacket(MemoryTracker * memory_tracker) + : mem_tracker_wrapper(memory_tracker) + {} + + void addChunk(std::string && value) + { + mem_tracker_wrapper.alloc(value.size()); + packet.add_chunks(std::move(value)); + } + + void serializeByResponse(const tipb::SelectResponse & response) + { + mem_tracker_wrapper.alloc(response.ByteSizeLong()); + if (!response.SerializeToString(packet.mutable_data())) + { + mem_tracker_wrapper.free(response.ByteSizeLong()); + throw Exception(fmt::format("Fail to serialize response, response size: {}", response.ByteSizeLong())); + } + } + + void read(const std::unique_ptr<::grpc::ClientAsyncReader<::mpp::MPPDataPacket>> & reader, void * callback) + { + reader->Read(&packet, callback); + need_recompute = true; + //we shouldn't update tracker now, since it's an async reader!! + } + + // we need recompute in some cases we can't update memory counter timely, such as async read + void recomputeTrackedMem() + { + if (need_recompute) + { + mem_tracker_wrapper.freeAll(); + mem_tracker_wrapper.alloc(estimateAllocatedSize(packet)); + need_recompute = false; + } + } + + bool read(const std::unique_ptr<::grpc::ClientReader<::mpp::MPPDataPacket>> & reader) + { + bool ret = reader->Read(&packet); + mem_tracker_wrapper.freeAll(); + mem_tracker_wrapper.alloc(estimateAllocatedSize(packet)); + return ret; + } + + void switchMemTracker(MemoryTracker * new_memory_tracker) + { + mem_tracker_wrapper.switchMemTracker(new_memory_tracker); + } + + bool hasError() const + { + return packet.has_error(); + } + + const ::mpp::Error & error() const + { + return packet.error(); + } + + mpp::MPPDataPacket & getPacket() + { + return packet; + } + + MemTrackerWrapper mem_tracker_wrapper; + mpp::MPPDataPacket packet; + bool need_recompute = false; +}; + +struct TrackedSelectResp +{ + explicit TrackedSelectResp() + : memory_tracker(current_memory_tracker) + {} + + void addChunk(std::string && value) + { + memory_tracker.alloc(value.size()); + auto * dag_chunk = response.add_chunks(); + dag_chunk->set_rows_data(std::move(value)); + } + + tipb::SelectResponse & getResponse() + { + return response; + } + + void setEncodeType(::tipb::EncodeType value) + { + response.set_encode_type(value); + } + + tipb::ExecutorExecutionSummary * addExecutionSummary() + { + return response.add_execution_summaries(); + } + + MemTrackerWrapper memory_tracker; + tipb::SelectResponse response; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp index ae720badb68..59409e5dd0b 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp @@ -77,11 +77,11 @@ struct MockLocalReader { while (true) { - MPPDataPacketPtr tmp_packet = local_sender->readForLocal(); + TrackedMppDataPacketPtr tmp_packet = local_sender->readForLocal(); bool success = tmp_packet != nullptr; if (success) { - write_packet_vec.push_back(tmp_packet->data()); + write_packet_vec.push_back(tmp_packet->packet.data()); } else { @@ -118,7 +118,7 @@ struct MockTerminateLocalReader void read() const { - MPPDataPacketPtr tmp_packet = local_sender->readForLocal(); + TrackedMppDataPacketPtr tmp_packet = local_sender->readForLocal(); local_sender->consumerFinish("Receiver closed"); } }; diff --git a/dbms/src/Flash/tests/bench_exchange.h b/dbms/src/Flash/tests/bench_exchange.h index d8300d45740..cb4d62aeebd 100644 --- a/dbms/src/Flash/tests/bench_exchange.h +++ b/dbms/src/Flash/tests/bench_exchange.h @@ -96,7 +96,7 @@ struct MockReceiverContext { // Not implement benchmark for Async GRPC for now. void init(UnaryCallback *) { assert(0); } - void read(MPPDataPacketPtr &, UnaryCallback *) { assert(0); } + void read(TrackedMppDataPacketPtr &, UnaryCallback *) { assert(0); } void finish(::grpc::Status &, UnaryCallback *) { assert(0); } }; diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 803d22251cd..eb45bd772d6 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -19,6 +19,10 @@ add_headers_and_sources(clickhouse_functions ./GatherUtils) add_headers_and_sources(clickhouse_functions ./Conditional) add_headers_and_sources(clickhouse_functions ${TiFlash_BINARY_DIR}/dbms/src/Functions) +if (TIFLASH_ENABLE_AVX_SUPPORT) + set_source_files_properties(CollationStringOptimized.cpp APPEND COMPILE_FLAGS "-mavx -mavx2") +endif () + list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp) list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h) diff --git a/dbms/src/Functions/CollationOperatorOptimized.h b/dbms/src/Functions/CollationOperatorOptimized.h index 3eed2ec2965..dc7ac843589 100644 --- a/dbms/src/Functions/CollationOperatorOptimized.h +++ b/dbms/src/Functions/CollationOperatorOptimized.h @@ -47,69 +47,17 @@ struct IsEqualRelated> static constexpr const bool value = true; }; -// Loop columns and invoke callback for each pair. -// Remove last zero byte. -template -FLATTEN_INLINE inline void LoopTwoColumns( +template +FLATTEN_INLINE static inline void LoopOneColumnCmpEqStr( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const ColumnString::Chars_t & b_data, - const ColumnString::Offsets & b_offsets, - size_t size, - F && func) -{ - ColumnString::Offset a_prev_offset = 0; - ColumnString::Offset b_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - auto a_size = a_offsets[i] - a_prev_offset; - auto b_size = b_offsets[i] - b_prev_offset; - - // Remove last zero byte. - func({reinterpret_cast(&a_data[a_prev_offset]), a_size - 1}, - {reinterpret_cast(&b_data[b_prev_offset]), b_size - 1}, - i); - - a_prev_offset = a_offsets[i]; - b_prev_offset = b_offsets[i]; - } -} - -// Loop one column and invoke callback for each pair. -// Remove last zero byte. -template -FLATTEN_INLINE inline void LoopOneColumn( - const ColumnString::Chars_t & a_data, - const ColumnString::Offsets & a_offsets, - size_t size, - F && func) -{ - ColumnString::Offset a_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - auto a_size = a_offsets[i] - a_prev_offset; - - // Remove last zero byte. - func({reinterpret_cast(&a_data[a_prev_offset]), a_size - 1}, i); - a_prev_offset = a_offsets[i]; - } -} - -template -FLATTEN_INLINE inline void LoopOneColumnCmpEqFixedStr( - const ColumnString::Chars_t & a_data, - const ColumnString::Offsets & a_offsets, - const char * src, - Result & c) + Result & c, + F && fn_is_eq) { LoopOneColumn(a_data, a_offsets, a_offsets.size(), [&](std::string_view view, size_t i) { if constexpr (trim) view = RightTrim(view); - auto res = 1; - if (view.size() == n) - res = mem_utils::memcmp_eq_fixed_size(view.data(), src) ? 0 : 1; + auto res = fn_is_eq(view) ? 0 : 1; c[i] = Op::apply(res, 0); }); } @@ -119,7 +67,7 @@ FLATTEN_INLINE inline void LoopOneColumnCmpEqFixedStr( // - Check if columns do NOT contain tail space // - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way template -ALWAYS_INLINE inline bool CompareStringVectorStringVector( +ALWAYS_INLINE inline bool CompareStringVectorStringVectorImpl( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, const ColumnString::Chars_t & b_data, @@ -179,16 +127,74 @@ ALWAYS_INLINE inline bool CompareStringVectorStringVector( return use_optimized_path; } +template +ALWAYS_INLINE static inline bool BinCollatorCompareStringVectorConstant(const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & tar_str_view, + Result & c) +{ + if constexpr (!IsEqualRelated::value) + return false; + +#ifdef M + static_assert(false, "`M` is defined"); +#endif +#define M(k) \ + case k: \ + { \ + LoopOneColumnCmpEqStr(a_data, a_offsets, c, [&](const std::string_view & src) -> bool { return (src.size() == (k)) && mem_utils::memcmp_eq_fixed_size<(k)>(src.data(), tar_str_view.data()); }); \ + return true; \ + } + if (likely(tar_str_view.size() <= 32)) + { + switch (tar_str_view.size()) + { + M(0); + M(1); + M(2); + M(3); + M(4); + M(5); + M(6); + M(7); + M(8); + M(9); + M(10); + M(11); + M(12); + M(13); + M(14); + M(15); + M(16); + default: + { + LoopOneColumnCmpEqStr(a_data, a_offsets, c, [&](const std::string_view & src) -> bool { + const size_t n = tar_str_view.size(); + if (src.size() != n) + return false; + const auto * p1 = src.data(); + const auto * p2 = tar_str_view.data(); + return mem_utils::memcmp_eq_fixed_size<16>(p1, p2) + && mem_utils::memcmp_eq_fixed_size<16>(p1 + n - 16, p2 + n - 16); + }); + return true; + } + } + } +#undef M + return false; +} + // Handle str-column compare const-str. // - Optimize bin collator // - Right trim const-str first // - Check if column does NOT contain tail space // - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way template -ALWAYS_INLINE inline bool CompareStringVectorConstant( +ALWAYS_INLINE static inline bool CompareStringVectorConstantImpl( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const std::string_view & b, + const std::string_view & _b, const TiDB::TiDBCollatorPtr & collator, Result & c) { @@ -199,44 +205,10 @@ ALWAYS_INLINE inline bool CompareStringVectorConstant( case TiDB::ITiDBCollator::CollatorType::LATIN1_BIN: case TiDB::ITiDBCollator::CollatorType::ASCII_BIN: { - std::string_view tar_str_view = RightTrim(b); // right trim const-str first + std::string_view tar_str_view = RightTrim(_b); // right trim const-str first - if constexpr (IsEqualRelated::value) - { -#ifdef M - static_assert(false, "`M` is defined"); -#endif -#define M(k) \ - case k: \ - { \ - LoopOneColumnCmpEqFixedStr(a_data, a_offsets, tar_str_view.data(), c); \ - return true; \ - } - - switch (tar_str_view.size()) - { - M(0); - M(1); - M(2); - M(3); - M(4); - M(5); - M(6); - M(7); - M(8); - M(9); - M(10); - M(11); - M(12); - M(13); - M(14); - M(15); - M(16); - default: - break; - } -#undef M - } + if (BinCollatorCompareStringVectorConstant(a_data, a_offsets, tar_str_view, c)) + return true; LoopOneColumn(a_data, a_offsets, a_offsets.size(), [&c, &tar_str_view](const std::string_view & view, size_t i) { if constexpr (IsEqualRelated::value) @@ -253,51 +225,19 @@ ALWAYS_INLINE inline bool CompareStringVectorConstant( } case TiDB::ITiDBCollator::CollatorType::BINARY: { - if constexpr (IsEqualRelated::value) - { -#ifdef M - static_assert(false, "`M` is defined"); -#endif -#define M(k) \ - case k: \ - { \ - LoopOneColumnCmpEqFixedStr(a_data, a_offsets, b.data(), c); \ - return true; \ - } + const std::string_view & tar_str_view = _b; // use original const-str - switch (b.size()) - { - M(0); - M(1); - M(2); - M(3); - M(4); - M(5); - M(6); - M(7); - M(8); - M(9); - M(10); - M(11); - M(12); - M(13); - M(14); - M(15); - M(16); - default: - break; - } -#undef M - } + if (BinCollatorCompareStringVectorConstant(a_data, a_offsets, tar_str_view, c)) + return true; - LoopOneColumn(a_data, a_offsets, a_offsets.size(), [&c, &b](const std::string_view & view, size_t i) { + LoopOneColumn(a_data, a_offsets, a_offsets.size(), [&c, &tar_str_view](const std::string_view & view, size_t i) { if constexpr (IsEqualRelated::value) { - c[i] = Op::apply(RawStrEqualCompare((view), b), 0); + c[i] = Op::apply(RawStrEqualCompare(view, tar_str_view), 0); } else { - c[i] = Op::apply(RawStrCompare((view), b), 0); + c[i] = Op::apply(RawStrCompare(view, tar_str_view), 0); } }); diff --git a/dbms/src/Functions/CollationStringComparision.h b/dbms/src/Functions/CollationStringComparision.h new file mode 100644 index 00000000000..78ffe0c0e60 --- /dev/null +++ b/dbms/src/Functions/CollationStringComparision.h @@ -0,0 +1,39 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +{ + +template +extern bool CompareStringVectorConstant( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & _b, + const TiDB::TiDBCollatorPtr & collator, + Result & c); + +template +extern bool CompareStringVectorStringVector( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + const TiDB::TiDBCollatorPtr & collator, + Result & c); + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Functions/CollationStringOptimized.cpp b/dbms/src/Functions/CollationStringOptimized.cpp new file mode 100644 index 00000000000..a25a44bccd0 --- /dev/null +++ b/dbms/src/Functions/CollationStringOptimized.cpp @@ -0,0 +1,132 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +#include +#include + +#ifdef TIFLASH_ENABLE_AVX_SUPPORT +#ifndef TIFLASH_USE_AVX2_COMPILE_FLAG +static_assert(false, "`dbms/src/Functions/CollationStringOptimized.cpp` need flag `-mavx2` to inline related functions"); +#endif +#endif + +namespace DB +{ + +template +bool StringPatternMatch( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & pattern_str, + uint8_t escape_char, + const TiDB::TiDBCollatorPtr & collator, + PaddedPODArray & c) +{ + return StringPatternMatchImpl(a_data, a_offsets, pattern_str, escape_char, collator, c); +} + +template bool StringPatternMatch( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & pattern_str, + uint8_t escape_char, + const TiDB::TiDBCollatorPtr & collator, + PaddedPODArray & c); + +template bool StringPatternMatch( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & pattern_str, + uint8_t escape_char, + const TiDB::TiDBCollatorPtr & collator, + PaddedPODArray & c); + + +#ifdef M +static_assert(false, "`M` is defined"); +#endif + +template +bool CompareStringVectorStringVector( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + const TiDB::TiDBCollatorPtr & collator, + Result & c) +{ + return CompareStringVectorStringVectorImpl(a_data, a_offsets, b_data, b_offsets, collator, c); +} + +#define M(OP, Column) \ + template bool CompareStringVectorStringVector /*NOLINT*/, PaddedPODArray>( \ + const ColumnString::Chars_t & a_data, \ + const ColumnString::Offsets & a_offsets, \ + const ColumnString::Chars_t & b_data, \ + const ColumnString::Offsets & b_offsets, \ + const TiDB::TiDBCollatorPtr & collator, \ + PaddedPODArray & c) + + +M(EqualsOp, ColumnUInt8); +M(NotEqualsOp, ColumnUInt8); +M(LessOp, ColumnUInt8); +M(GreaterOp, ColumnUInt8); +M(LessOrEqualsOp, ColumnUInt8); +M(GreaterOrEqualsOp, ColumnUInt8); +M(ReversedCmpOp, ColumnInt8); +M(CmpOp, ColumnInt8); + +#undef M + +template +bool CompareStringVectorConstant( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & _b, + const TiDB::TiDBCollatorPtr & collator, + Result & c) +{ + return CompareStringVectorConstantImpl(a_data, a_offsets, _b, collator, c); +} + +#define M(OP, Column) \ + template bool CompareStringVectorConstant /*NOLINT*/, PaddedPODArray>( \ + const ColumnString::Chars_t & a_data, \ + const ColumnString::Offsets & a_offsets, \ + const std::string_view & _b, \ + const TiDB::TiDBCollatorPtr & collator, \ + PaddedPODArray & c) + + +M(EqualsOp, ColumnUInt8); +M(NotEqualsOp, ColumnUInt8); +M(LessOp, ColumnUInt8); +M(GreaterOp, ColumnUInt8); +M(LessOrEqualsOp, ColumnUInt8); +M(GreaterOrEqualsOp, ColumnUInt8); +M(ReversedCmpOp, ColumnInt8); +M(CmpOp, ColumnInt8); + +#undef M + + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Functions/CollationStringSearch.h b/dbms/src/Functions/CollationStringSearch.h new file mode 100644 index 00000000000..9b3e9001094 --- /dev/null +++ b/dbms/src/Functions/CollationStringSearch.h @@ -0,0 +1,31 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +{ + +template +extern bool StringPatternMatch( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & pattern_str, + uint8_t escape_char, + const TiDB::TiDBCollatorPtr & collator, + PaddedPODArray & c); + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Functions/CollationStringSearchOptimized.h b/dbms/src/Functions/CollationStringSearchOptimized.h index a986290519e..bf6574f648f 100644 --- a/dbms/src/Functions/CollationStringSearchOptimized.h +++ b/dbms/src/Functions/CollationStringSearchOptimized.h @@ -14,11 +14,9 @@ #pragma once -#include #include -#include #include -#include +#include #include #include @@ -88,7 +86,7 @@ struct BinStrPattern auto last_match_start = std::string_view::npos; - const auto & fn_try_add_last_match_str = [&](size_t end_offset) { + auto && fn_try_add_last_match_str = [&](size_t end_offset) { if (last_match_start != std::string_view::npos) { match_sub_str.emplace_back(&pattern[last_match_start], end_offset - last_match_start); @@ -194,7 +192,7 @@ struct BinStrPattern // check str equality // - make src invalid if remain size if smaller than required - bool matchStrEqual(const std::string_view & src, MatchDesc & desc) const + ALWAYS_INLINE inline bool matchStrEqual(const std::string_view & src, MatchDesc & desc) const { const auto & match_str = match_sub_str[desc.match_str_index_start]; if (desc.srcSize() < match_str.size()) @@ -202,7 +200,7 @@ struct BinStrPattern desc.makeSrcInvalid(); return false; } - if (DB::RawStrEqualCompare(desc.getSrcStrView(src.data(), match_str.size()), match_str)) + if (!mem_utils::IsStrViewEqual(desc.getSrcStrView(src.data(), match_str.size()), match_str)) { return false; } @@ -214,7 +212,7 @@ struct BinStrPattern // match from start exactly // - return true if meet % // - return false if failed to match else true - bool matchExactly(const std::string_view & src, MatchDesc & cur_match_desc) const + ALWAYS_INLINE inline bool matchExactly(const std::string_view & src, MatchDesc & cur_match_desc) const { // match from start for (; !cur_match_desc.patternEmpty(); cur_match_desc.pattern_index_start++) @@ -245,7 +243,7 @@ struct BinStrPattern // match from end exactly // - return true if meet % // - return false if failed to match else true - bool matchExactlyReverse(const std::string_view & src, MatchDesc & cur_match_desc) const + ALWAYS_INLINE inline bool matchExactlyReverse(const std::string_view & src, MatchDesc & cur_match_desc) const { for (; !cur_match_desc.patternEmpty(); --cur_match_desc.pattern_index_end) { @@ -263,7 +261,7 @@ struct BinStrPattern return false; } - if (DB::RawStrEqualCompare({src.data() + cur_match_desc.src_index_end - match_str.size(), match_str.size()}, match_str)) + if (!mem_utils::IsStrViewEqual({src.data() + cur_match_desc.src_index_end - match_str.size(), match_str.size()}, match_str)) { return false; } @@ -286,7 +284,7 @@ struct BinStrPattern // search by pattern `...%..%` // - return true if meet % // - return false if failed to search - bool searchByPattern(const std::string_view & src, MatchDesc & desc) const + ALWAYS_INLINE inline bool searchByPattern(const std::string_view & src, MatchDesc & desc) const { assert(match_types[desc.pattern_index_end - 1] == MatchType::Any); assert(!desc.patternEmpty()); @@ -320,10 +318,8 @@ struct BinStrPattern // search sub str // - seachers like `ASCIICaseSensitiveStringSearcher` or `Volnitsky` are too heavy for small str - // - TODO: optimize strstr search by simd { - pos = src_view.find(match_str); - // pos = sse2_strstr(src_view, match_str); + pos = mem_utils::StrFind(src_view, match_str); } if (pos == std::string_view::npos) @@ -356,7 +352,7 @@ struct BinStrPattern } }; - bool match(std::string_view src) const + ALWAYS_INLINE inline bool match(std::string_view src) const { MatchDesc cur_match_desc; { @@ -421,7 +417,7 @@ struct BinStrPattern namespace DB { template -ALWAYS_INLINE inline void BinStringPatternMatch( +inline void BinStringPatternMatch( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, const std::string_view & pattern_str, @@ -436,7 +432,7 @@ ALWAYS_INLINE inline void BinStringPatternMatch( } template -ALWAYS_INLINE inline bool StringPatternMatch( +ALWAYS_INLINE inline bool StringPatternMatchImpl( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, const std::string_view & pattern_str, diff --git a/dbms/src/Functions/FunctionsComparison.cpp b/dbms/src/Functions/FunctionsComparison.cpp index b97fb7c5379..bea75b881ff 100644 --- a/dbms/src/Functions/FunctionsComparison.cpp +++ b/dbms/src/Functions/FunctionsComparison.cpp @@ -108,7 +108,7 @@ void FunctionComparison::executeTupleImp } template <> -void FunctionComparison::executeTupleImpl( +void FunctionComparison::executeTupleImpl( [[maybe_unused]] Block & block, [[maybe_unused]] size_t result, [[maybe_unused]] const ColumnsWithTypeAndName & x, diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 51ce778efd7..84364e74369 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include #include @@ -558,7 +558,8 @@ struct NameStrcmp template < template class Op, - typename Name> + typename Name, + typename StrCmpRetColType = ColumnUInt8> class FunctionComparison : public IFunction { public: @@ -803,7 +804,6 @@ class FunctionComparison : public IFunction friend class FunctionStrcmp; - template bool executeString(Block & block, size_t result, const IColumn * c0, const IColumn * c1) const { const auto * c0_string = checkAndGetColumn(c0); @@ -815,9 +815,9 @@ class FunctionComparison : public IFunction return false; if (collator != nullptr) - return executeStringWithCollator(block, result, c0, c1, c0_string, c1_string, c0_const, c1_const); + return executeStringWithCollator(block, result, c0, c1, c0_string, c1_string, c0_const, c1_const); else - return executeStringWithoutCollator(block, result, c0, c1, c0_string, c1_string, c0_const, c1_const); + return executeStringWithoutCollator(block, result, c0, c1, c0_string, c1_string, c0_const, c1_const); } void executeDateOrDateTimeOrEnumWithConstString( @@ -1299,7 +1299,8 @@ class FunctionComparison : public IFunction } }; -class FunctionStrcmp : public FunctionComparison +using StrcmpReturnColumnType = ColumnInt8; +class FunctionStrcmp : public FunctionComparison { public: static FunctionPtr create(const Context &) { return std::make_shared(); }; @@ -1309,7 +1310,7 @@ class FunctionStrcmp : public FunctionComparison const IColumn * col_left_untyped = block.getByPosition(arguments[0]).column.get(); const IColumn * col_right_untyped = block.getByPosition(arguments[1]).column.get(); - bool success = executeString(block, result, col_left_untyped, col_right_untyped); + bool success = executeString(block, result, col_left_untyped, col_right_untyped); if (!success) { throw Exception( diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 5da3ee55e60..7278e61dfce 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -17,13 +17,14 @@ #include #include #include -#include +#include #include #include #include #include #include #include +#include #include #include diff --git a/dbms/src/Functions/tests/gtest_strings_cmp.cpp b/dbms/src/Functions/tests/gtest_strings_cmp.cpp index 7e94a7ed30e..0dc85854ab6 100644 --- a/dbms/src/Functions/tests/gtest_strings_cmp.cpp +++ b/dbms/src/Functions/tests/gtest_strings_cmp.cpp @@ -70,6 +70,16 @@ try ASSERT_COLUMN_EQ(createColumn({-1}), executeFunction("strcmp", {createColumn({"A"}), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); ASSERT_COLUMN_EQ(createColumn({-1, 0, 1, 1}), executeFunction("strcmp", {createColumn({"A", "B", "C", "D"}), createConstColumn>(4, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + { + std::string src(35, char(2)); + std::string tar1 = src; + std::string tar2 = tar1; + std::string tar3 = tar1; + tar2.back() = char(1); + tar3.back() = char(3); + ASSERT_COLUMN_EQ(createColumn({-1, 0, 1}), executeFunction("strcmp", {createColumn({tar2, tar1, tar3}), createConstColumn>(3, src)}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::BINARY))); + } + // constant with column ASSERT_COLUMN_EQ(createColumn>({1, 0, -1, std::nullopt}), executeFunction("strcmp", {createConstColumn>(4, "b"), createColumn>({"a", "b", "c", std::nullopt})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); ASSERT_COLUMN_EQ(createColumn>({1, 0, -1, std::nullopt}), executeFunction("strcmp", {createConstColumn>(4, "B"), createColumn>({"a", "b", "c", std::nullopt})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); @@ -92,5 +102,69 @@ try } } CATCH + +TEST_F(Strcmp, StrEq) +try +{ + const std::string equals_fn_name = "equals"; + for (const auto col_id : { + TiDB::ITiDBCollator::UTF8MB4_BIN, + TiDB::ITiDBCollator::BINARY, + }) + { + const auto * collator = TiDB::ITiDBCollator::getCollator(col_id); + + // with collation + { + // column : column + ASSERT_COLUMN_EQ(createColumn({0, 0, 1, 1, 0}), executeFunction(equals_fn_name, {createColumn({"a", "b", "a", "", "00"}), createColumn({"b", "a", "a", "", "0"})}, collator)); + // column : constant + // constant : column + { + for (size_t size = 1; size < 35; ++size) + { + std::string a(size, '0'); + std::string b(size, '0'); + b.back() = '1'; + ASSERT_COLUMN_EQ(createColumn({1, 0}), executeFunction(equals_fn_name, {createConstColumn(2, a), createColumn({a, b})}, collator)); + ASSERT_COLUMN_EQ(createColumn({1, 0}), executeFunction(equals_fn_name, {createColumn({a, b}), createConstColumn(2, a)}, collator)); + { + auto c = a; + a += " "; + b += " "; + c += " "; + if (!collator->isBinary()) + { + ASSERT_COLUMN_EQ(createColumn({1, 0}), executeFunction(equals_fn_name, {createConstColumn(2, c), createColumn({a, b})}, collator)); + ASSERT_COLUMN_EQ(createColumn({1, 0}), executeFunction(equals_fn_name, {createColumn({a, b}), createConstColumn(2, c)}, collator)); + } + else + { + ASSERT_COLUMN_EQ(createColumn({0, 0, 1}), executeFunction(equals_fn_name, {createConstColumn(3, c), createColumn({a, b, c})}, collator)); + ASSERT_COLUMN_EQ(createColumn({0, 0, 1}), executeFunction(equals_fn_name, {createColumn({a, b, c}), createConstColumn(3, c)}, collator)); + } + } + } + } + + // column : nullable + ASSERT_COLUMN_EQ(createColumn>({0, 0, 1, std::nullopt, std::nullopt}), executeFunction(equals_fn_name, {createColumn>({"1", "123", "45", "123", std::nullopt}), createColumn>({"123", "1", "45", std::nullopt, "123"})}, collator)); + + // column with constant + ASSERT_COLUMN_EQ(createColumn>({0, 1, std::nullopt}), executeFunction(equals_fn_name, {createColumn>({"aa", "bb", std::nullopt}), createConstColumn>(3, "bb")}, collator)); + + // constant with column + ASSERT_COLUMN_EQ(createColumn>({0, 1, std::nullopt}), executeFunction(equals_fn_name, {createConstColumn>(3, "bb"), createColumn>({"aa", "bb", std::nullopt})}, collator)); + + // constant with nullable + ASSERT_COLUMN_EQ(createColumn>({0}), executeFunction(equals_fn_name, {createColumn({"a"}), createColumn>({"b"})}, collator)); + ASSERT_COLUMN_EQ(createColumn>({std::nullopt}), executeFunction(equals_fn_name, {createColumn({"a"}), createColumn>({std::nullopt})}, collator)); + } + } + { + ASSERT_COLUMN_EQ(createColumn>({0, 1, std::nullopt}), executeFunction(equals_fn_name, {createConstColumn(3, ""), createColumn>({"123", "", std::nullopt})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_BIN))); + } +} +CATCH } // namespace tests } // namespace DB diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 0f667cfd396..5e50d560bb0 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -162,6 +162,7 @@ ProcessList::EntryPtr ProcessList::insert( /// not for specific users, sessions or queries, /// because this setting is effectively global. total_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_all_queries); + total_memory_tracker.setBytesThatRssLargerThanLimit(settings.bytes_that_rss_larger_than_limit); total_memory_tracker.setDescription("(total)"); user_process_list.user_memory_tracker.setNext(&total_memory_tracker); } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 354c03a9411..52fec1c4c1a 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -317,8 +317,6 @@ struct Settings \ M(SettingDouble, dt_page_gc_threshold, 0.5, "Max valid rate of deciding to do a GC in PageStorage") \ M(SettingBool, dt_enable_read_thread, false, "Enable storage read thread or not") \ - M(SettingDouble, dt_block_slots_scale, 1.0, "Block slots limit of a read request") \ - M(SettingDouble, dt_active_segments_scale, 1.0, "Acitve segments limit of a read request") \ \ M(SettingChecksumAlgorithm, dt_checksum_algorithm, ChecksumAlgo::XXH3, "Checksum algorithm for delta tree stable storage") \ M(SettingCompressionMethod, dt_compression_method, CompressionMethod::LZ4, "The method of data compression when writing.") \ @@ -343,6 +341,7 @@ struct Settings M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.") \ M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.") \ M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.") \ + M(SettingUInt64, bytes_that_rss_larger_than_limit, 5368709120, "How many bytes RSS(Resident Set Size) can be larger than limit(max_memory_usage_for_all_queries). Default: 5GB ") \ \ M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.") \ M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.") \ diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 77a3a76d842..fc2103d7838 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -22,6 +22,8 @@ #include #include #include +#include +#include #include #include #include @@ -231,6 +233,16 @@ std::tuple executeQueryImpl( context.setProcessListElement(&process_list_entry->get()); } + // Do set-up work for tunnels and receivers after ProcessListEntry is constructed, + // so that we can propagate current_memory_tracker into them. + if (context.getDAGContext()) // When using TiFlash client, dag context will be nullptr in this case. + { + if (context.getDAGContext()->tunnel_set) + context.getDAGContext()->tunnel_set->updateMemTracker(); + if (context.getDAGContext()->getMppReceiverSet()) + context.getDAGContext()->getMppReceiverSet()->setUpConnection(); + } + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_interpreter_failpoint); auto interpreter = query_src.interpreter(context, stage); res = interpreter->execute(); diff --git a/dbms/src/Server/FlashGrpcServerHolder.cpp b/dbms/src/Server/FlashGrpcServerHolder.cpp index 1190985004d..c359db24298 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.cpp +++ b/dbms/src/Server/FlashGrpcServerHolder.cpp @@ -84,6 +84,7 @@ FlashGrpcServerHolder::FlashGrpcServerHolder(Context & context, Poco::Util::Laye : log(log_) , is_shutdown(std::make_shared>(false)) { + background_task.begin(); grpc::ServerBuilder builder; if (security_config.has_tls_config) { @@ -190,6 +191,7 @@ FlashGrpcServerHolder::~FlashGrpcServerHolder() LOG_FMT_INFO(log, "Begin to shut down flash service"); flash_service.reset(); LOG_FMT_INFO(log, "Shut down flash service"); + background_task.end(); } catch (...) { diff --git a/dbms/src/Server/FlashGrpcServerHolder.h b/dbms/src/Server/FlashGrpcServerHolder.h index 57146f40aae..054138de246 100644 --- a/dbms/src/Server/FlashGrpcServerHolder.h +++ b/dbms/src/Server/FlashGrpcServerHolder.h @@ -13,6 +13,7 @@ // limitations under the License. #pragma once +#include #include #include #include @@ -49,6 +50,7 @@ class FlashGrpcServerHolder std::vector> cqs; std::vector> notify_cqs; std::shared_ptr thread_manager; + CollectProcInfoBackgroundTask background_task; }; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index d2244cae7d5..900239e623a 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1242,7 +1242,7 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, const RSOperatorPtr & filter, const String & tracing_id, bool keep_order, - bool is_fast_mode, + bool is_fast_scan, size_t expected_block_size, const SegmentIdSet & read_segments, size_t extra_table_id_index) @@ -1277,8 +1277,8 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, filter, max_version, expected_block_size, - /* is_raw = */ is_fast_mode, - /* do_delete_mark_filter_for_raw = */ is_fast_mode, + /* is_raw = */ is_fast_scan, + /* do_delete_mark_filter_for_raw = */ is_fast_scan, std::move(tasks), after_segment_read); @@ -1308,8 +1308,8 @@ BlockInputStreams DeltaMergeStore::read(const Context & db_context, filter, max_version, expected_block_size, - /* is_raw_ */ is_fast_mode, - /* do_delete_mark_filter_for_raw_ */ is_fast_mode, + /* is_raw_= */ is_fast_scan, + /* do_delete_mark_filter_for_raw_= */ is_fast_scan, extra_table_id_index, physical_table_id, req_info); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 4013cd22c90..769786ce070 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -359,9 +359,9 @@ class DeltaMergeStore : private boost::noncopyable /// Read rows in two modes: - /// when is_fast_mode == false, we are in normal mode. Thus we will read rows with MVCC filtering, del mark !=0 filter and sorted merge - /// when is_fast_mode == true, we are in fast mode. Thus we will read rows without MVCC and sorted merge - /// `sorted_ranges` should be already sorted and merged + /// when is_fast_scan == false, we will read rows with MVCC filtering, del mark !=0 filter and sorted merge. + /// when is_fast_scan == true, we will read rows without MVCC and sorted merge. + /// `sorted_ranges` should be already sorted and merged. BlockInputStreams read(const Context & db_context, const DB::Settings & db_settings, const ColumnDefines & columns_to_read, @@ -371,7 +371,7 @@ class DeltaMergeStore : private boost::noncopyable const RSOperatorPtr & filter, const String & tracing_id, bool keep_order, - bool is_fast_mode = false, // set true when read in fast mode + bool is_fast_scan = false, size_t expected_block_size = DEFAULT_BLOCK_SIZE, const SegmentIdSet & read_segments = {}, size_t extra_table_id_index = InvalidColumnID); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp index f8f73574b4b..bfb5d2cc054 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.cpp @@ -64,7 +64,7 @@ DMFileBlockInputStreamPtr DMFileBlockInputStreamBuilder::build(const DMFilePtr & is_common_handle, enable_handle_clean_read, enable_del_clean_read, - is_fast_mode, + is_fast_scan, max_data_version, std::move(pack_filter), mark_cache, diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.h b/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.h index ce2d06f2410..e1552994396 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileBlockInputStream.h @@ -83,20 +83,19 @@ class DMFileBlockInputStreamBuilder // **** filters **** // // Only set enable_handle_clean_read_ param to true when - // in normal mode: - // 1. There is no delta. - // 2. You don't need pk, version and delete_tag columns - // in fast mode: - // 1. You don't need pk columns - // If you have no idea what it means, then simply set it to false. - // Only set is_fast_mode_ param to true when read in fast mode. - // Only set enable_del_clean_read_ param to true when you don't need del columns in fast mode. + // in normal mode (is_fast_scan_ == false): + // 1. There is no delta. + // 2. You don't need pk, version and delete_tag columns + // in fast scan mode (is_fast_scan_ == true): + // 1. You don't need pk columns + // If you have no idea what it means, then simply set it to false. + // Only set enable_del_clean_read_ param to true when you don't need del columns in fast scan. // `max_data_version_` is the MVCC filter version for reading. Used by clean read check - DMFileBlockInputStreamBuilder & enableCleanRead(bool enable_handle_clean_read_, bool is_fast_mode_, bool enable_del_clean_read_, UInt64 max_data_version_) + DMFileBlockInputStreamBuilder & enableCleanRead(bool enable_handle_clean_read_, bool is_fast_scan_, bool enable_del_clean_read_, UInt64 max_data_version_) { enable_handle_clean_read = enable_handle_clean_read_; enable_del_clean_read = enable_del_clean_read_; - is_fast_mode = is_fast_mode_; + is_fast_scan = is_fast_scan_; max_data_version = max_data_version_; return *this; } @@ -159,8 +158,9 @@ class DMFileBlockInputStreamBuilder FileProviderPtr file_provider; // clean read + bool enable_handle_clean_read = false; - bool is_fast_mode = false; + bool is_fast_scan = false; bool enable_del_clean_read = false; UInt64 max_data_version = std::numeric_limits::max(); // Rough set filter diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp index c8e425453c9..19ddf66c015 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.cpp @@ -210,7 +210,7 @@ DMFileReader::DMFileReader( // clean read bool enable_handle_clean_read_, bool enable_del_clean_read_, - bool is_fast_mode_, + bool is_fast_scan_, UInt64 max_read_version_, // filters DMFilePackFilter && pack_filter_, @@ -233,7 +233,7 @@ DMFileReader::DMFileReader( , single_file_mode(dmfile_->isSingleFileMode()) , enable_handle_clean_read(enable_handle_clean_read_) , enable_del_clean_read(enable_del_clean_read_) - , is_fast_mode(is_fast_mode_) + , is_fast_scan(is_fast_scan_) , max_read_version(max_read_version_) , pack_filter(std::move(pack_filter_)) , skip_packs_by_column(read_columns.size(), 0) @@ -364,10 +364,11 @@ Block DMFileReader::read() } // TODO: this will need better algorithm: we should separate those packs which can and can not do clean read. - bool do_clean_read_on_normal_mode = enable_handle_clean_read && expected_handle_res == All && not_clean_rows == 0 && (!is_fast_mode); + bool do_clean_read_on_normal_mode = enable_handle_clean_read && expected_handle_res == All && not_clean_rows == 0 && (!is_fast_scan); + + bool do_clean_read_on_handle_on_fast_mode = enable_handle_clean_read && is_fast_scan && expected_handle_res == All; + bool do_clean_read_on_del_on_fast_mode = enable_del_clean_read && is_fast_scan && deleted_rows == 0; - bool do_clean_read_on_handle_on_fast_mode = enable_handle_clean_read && is_fast_mode && expected_handle_res == All; - bool do_clean_read_on_del_on_fast_mode = enable_del_clean_read && is_fast_mode && deleted_rows == 0; if (do_clean_read_on_normal_mode) { diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileReader.h b/dbms/src/Storages/DeltaMerge/File/DMFileReader.h index 76adef0301a..2eb14c6d19e 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileReader.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileReader.h @@ -78,7 +78,7 @@ class DMFileReader // If you have no idea what it means, then simply set it to false. bool enable_handle_clean_read_, bool enable_del_clean_read_, - bool is_fast_mode_, + bool is_fast_scan_, // The the MVCC filter version. Used by clean read check. UInt64 max_read_version_, // filters @@ -148,7 +148,8 @@ class DMFileReader // if we don't need del column, we will try to do clean read on del_column(enable_del_clean_read is true). const bool enable_handle_clean_read; const bool enable_del_clean_read; - const bool is_fast_mode; + const bool is_fast_scan; + const UInt64 max_read_version; /// Filters diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index e706b51a767..3742a18aff2 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -534,6 +534,8 @@ BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, /// But this way seems not to be robustness enough, maybe we need another flag? auto new_columns_to_read = std::make_shared(); + // new_columns_to_read need at most columns_to_read.size() + 2, due to may extra insert into the handle column and del_mark column. + new_columns_to_read->reserve(columns_to_read.size() + 2); new_columns_to_read->push_back(getExtraHandleColumnDefine(is_common_handle)); if (filter_delete_mark) @@ -578,7 +580,7 @@ BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, std::numeric_limits::max(), expected_block_size, /* enable_handle_clean_read */ enable_handle_clean_read, - /* is_fast_mode */ filter_delete_mark, + /* is_fast_scan */ filter_delete_mark, /* enable_del_clean_read */ enable_del_clean_read); BlockInputStreamPtr delta_stream = std::make_shared(dm_context, segment_snap->delta, new_columns_to_read, this->rowkey_range); diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.cpp b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.cpp index 80ff8487c21..88dc0742c89 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.cpp +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTaskPool.cpp @@ -222,8 +222,7 @@ int64_t SegmentReadTaskPool::decreaseUnorderedInputStreamRefCount() int64_t SegmentReadTaskPool::getFreeBlockSlots() const { - double block_slots_scale = dm_context->db_context.getSettingsRef().dt_block_slots_scale; - auto block_slots = static_cast(std::ceil(unordered_input_stream_ref_count.load(std::memory_order_relaxed) * block_slots_scale)); + auto block_slots = unordered_input_stream_ref_count.load(std::memory_order_relaxed); if (block_slots < 3) { block_slots = 3; @@ -233,8 +232,7 @@ int64_t SegmentReadTaskPool::getFreeBlockSlots() const int64_t SegmentReadTaskPool::getFreeActiveSegmentCountUnlock() { - double active_segments_scale = dm_context->db_context.getSettingsRef().dt_active_segments_scale; - auto active_segment_limit = static_cast(std::ceil(unordered_input_stream_ref_count.load(std::memory_order_relaxed) * active_segments_scale)); + auto active_segment_limit = unordered_input_stream_ref_count.load(std::memory_order_relaxed); if (active_segment_limit < 2) { active_segment_limit = 2; diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 3a968ed1dfa..9827655a265 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -330,17 +330,17 @@ StableValueSpace::Snapshot::getInputStream( UInt64 max_data_version, size_t expected_block_size, bool enable_handle_clean_read, - bool is_fast_mode, + bool is_fast_scan, bool enable_del_clean_read) { - LOG_FMT_DEBUG(log, "max_data_version: {}, enable_handle_clean_read: {}, is_fast_mode: {}, enable_del_clean_read: {}", max_data_version, enable_handle_clean_read, is_fast_mode, enable_del_clean_read); + LOG_FMT_DEBUG(log, "max_data_version: {}, enable_handle_clean_read: {}, is_fast_mode: {}, enable_del_clean_read: {}", max_data_version, enable_handle_clean_read, is_fast_scan, enable_del_clean_read); SkippableBlockInputStreams streams; for (size_t i = 0; i < stable->files.size(); i++) { DMFileBlockInputStreamBuilder builder(context.db_context); builder - .enableCleanRead(enable_handle_clean_read, is_fast_mode, enable_del_clean_read, max_data_version) + .enableCleanRead(enable_handle_clean_read, is_fast_scan, enable_del_clean_read, max_data_version) .setRSOperator(filter) .setColumnCache(column_caches[i]) .setTracingID(context.tracing_id) diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.h b/dbms/src/Storages/DeltaMerge/StableValueSpace.h index f254a8c21a4..1ab6449dc01 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.h @@ -151,7 +151,7 @@ class StableValueSpace : public std::enable_shared_from_this UInt64 max_data_version, size_t expected_block_size, bool enable_handle_clean_read, - bool is_fast_mode = false, + bool is_fast_scan = false, bool enable_del_clean_read = false); RowsAndBytes getApproxRowsAndBytes(const DMContext & context, const RowKeyRange & range) const; diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool.cpp index 79206bd4d08..c9ba3943a60 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool.cpp @@ -544,10 +544,6 @@ void StoragePool::dataRegisterExternalPagesCallbacks(const ExternalPageCallbacks break; } case PageStorageRunMode::ONLY_V3: - { - data_storage_v3->registerExternalPagesCallbacks(callbacks); - break; - } case PageStorageRunMode::MIX_MODE: { // We have transformed all pages from V2 to V3 in `restore`, so @@ -570,13 +566,10 @@ void StoragePool::dataUnregisterExternalPagesCallbacks(NamespaceId ns_id) break; } case PageStorageRunMode::ONLY_V3: - { - data_storage_v3->unregisterExternalPagesCallbacks(ns_id); - break; - } case PageStorageRunMode::MIX_MODE: { - // no need unregister callback in V2. + // We have transformed all pages from V2 to V3 in `restore`, so + // only need to unregister callbacks for V3. data_storage_v3->unregisterExternalPagesCallbacks(ns_id); break; } diff --git a/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt b/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt index 44ca6ccb367..1e52939a46c 100644 --- a/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt +++ b/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt @@ -42,8 +42,8 @@ target_link_libraries(dm_test_storage_delta_merge add_executable(dm_test_delta_merge_store EXCLUDE_FROM_ALL gtest_dm_delta_merge_store.cpp) target_link_libraries(dm_test_delta_merge_store dbms gtest_main clickhouse_functions) -add_executable(dm_test_delta_merge_store_for_fast_mode EXCLUDE_FROM_ALL gtest_dm_delta_merge_store_for_fast_mode.cpp) -target_link_libraries(dm_test_delta_merge_store_for_fast_mode dbms gtest_main clickhouse_functions) +add_executable(dm_test_delta_merge_store_for_fast_scan EXCLUDE_FROM_ALL gtest_dm_delta_merge_store_for_fast_scan.cpp) +target_link_libraries(dm_test_delta_merge_store_for_fast_scan dbms gtest_main clickhouse_functions) add_executable(dm_test_segment EXCLUDE_FROM_ALL gtest_dm_segment.cpp) target_link_libraries(dm_test_segment dbms gtest_main clickhouse_functions) diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index 1e48dd4e4af..0259d228038 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -259,7 +259,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -358,7 +358,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_NROWS(in, 0); } @@ -437,7 +437,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -486,7 +486,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -513,7 +513,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -588,7 +588,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -663,7 +663,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -680,11 +680,11 @@ try columns, {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, /* num_streams= */ 1, - /* max_version= */ UInt64(1), + /* max_version= */ static_cast(1), EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -729,7 +729,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -759,7 +759,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -802,7 +802,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -821,7 +821,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -840,7 +840,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -859,7 +859,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -916,7 +916,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -942,7 +942,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -968,7 +968,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -987,7 +987,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -1008,7 +1008,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -1065,7 +1065,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; @@ -1091,7 +1091,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; @@ -1117,7 +1117,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; @@ -1169,7 +1169,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; @@ -1195,7 +1195,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1); BlockInputStreamPtr in = ins[0]; @@ -1277,7 +1277,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; @@ -1366,7 +1366,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; @@ -1455,7 +1455,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; @@ -1531,7 +1531,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; @@ -1615,7 +1615,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( @@ -1681,7 +1681,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( @@ -1747,7 +1747,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -1812,7 +1812,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -1877,7 +1877,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( @@ -1941,7 +1941,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; std::vector datetime_data( @@ -2011,7 +2011,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -2091,7 +2091,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; @@ -2208,7 +2208,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; @@ -2256,7 +2256,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr & in = ins[0]; @@ -2342,7 +2342,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -2384,7 +2384,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // FIXME!!! @@ -2495,7 +2495,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // mock common handle @@ -2597,7 +2597,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { @@ -2669,7 +2669,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { @@ -2694,11 +2694,11 @@ try columns, {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, /* num_streams= */ 1, - /* max_version= */ UInt64(1), + /* max_version= */ static_cast(1), EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { @@ -2754,7 +2754,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // mock common handle auto common_handle_coldata = []() { @@ -2791,7 +2791,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; // mock common handle, data range after deletion is [64, 128) auto common_handle_coldata = []() { @@ -2858,7 +2858,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024); ASSERT_EQ(ins.size(), 1UL); BlockInputStreamPtr in = ins[0]; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_for_fast_mode.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_for_fast_scan.cpp similarity index 96% rename from dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_for_fast_mode.cpp rename to dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_for_fast_scan.cpp index 86499ec4fbc..42736b3d893 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_for_fast_mode.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_for_fast_scan.cpp @@ -27,7 +27,7 @@ namespace DM { namespace tests { -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithoutRangeFilter) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithOnlyInsertWithoutRangeFilter) { /// test under only insert data (no update, no delete) with all range @@ -87,7 +87,7 @@ TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithoutRangeFilter) EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -100,7 +100,7 @@ TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithoutRangeFilter) } } -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithRangeFilter) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithOnlyInsertWithRangeFilter) { /// test under only insert data (no update, no delete) with range filter @@ -171,7 +171,7 @@ TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithRangeFilter) EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -184,7 +184,7 @@ TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithRangeFilter) } } -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithMultipleBlockWithoutFlushCache) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithOnlyInsertWithMultipleBlockWithoutFlushCache) try { const size_t num_write_rows = 32; @@ -246,7 +246,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; switch (mode) { @@ -303,7 +303,7 @@ try } CATCH -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithMultipleBlockWithoutCompact) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithOnlyInsertWithMultipleBlockWithoutCompact) try { const size_t num_write_rows = 32; @@ -367,7 +367,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; switch (mode) { @@ -408,7 +408,7 @@ try } CATCH -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithMultipleBlockWithCompact) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithOnlyInsertWithMultipleBlockWithCompact) try { const size_t num_write_rows = 32; @@ -473,7 +473,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; switch (mode) { @@ -514,7 +514,7 @@ try } CATCH -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithOnlyInsertWithMultipleBlockWithCompactAndMergeDelta) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithOnlyInsertWithMultipleBlockWithCompactAndMergeDelta) try { const size_t num_write_rows = 32; @@ -582,7 +582,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -593,7 +593,7 @@ try CATCH // Insert + Update -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithMultipleBlockWithOverlap) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithMultipleBlockWithOverlap) try { const size_t num_write_rows = 32; @@ -661,7 +661,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; switch (mode) @@ -732,7 +732,7 @@ try CATCH // Insert + Delete row -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithDeleteRow) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithDeleteRow) try { const ColumnDefine col_str_define(2, "col2", std::make_shared()); @@ -808,7 +808,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; // filter del mark = 1, thus just read the insert data before delete ASSERT_INPUTSTREAM_COLS_UR( @@ -834,7 +834,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -845,7 +845,7 @@ try CATCH // Insert + Delete Range -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithDeleteRange) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithDeleteRange) try { const size_t num_rows_write = 128; @@ -880,7 +880,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -905,7 +905,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order = */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; // filter del mark = 1, thus just read the insert data before delete ASSERT_INPUTSTREAM_COLS_UR( @@ -916,7 +916,7 @@ try } CATCH -TEST_P(DeltaMergeStoreRWTest, TestFastModeWithDeleteWithMergeDelta) +TEST_P(DeltaMergeStoreRWTest, TestFastScanWithDeleteWithMergeDelta) try { const size_t num_rows_write = 128; @@ -965,7 +965,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; auto pk_coldata = createNumbers(num_deleted_rows, num_rows_write); ASSERT_EQ(pk_coldata.size(), num_rows_write - num_deleted_rows); @@ -978,7 +978,7 @@ try CATCH // insert + update + delete and fast mode first and then normal mode, to check the mode conversion is ok -TEST_P(DeltaMergeStoreRWTest, TestFastModeComplexWithModeConversion) +TEST_P(DeltaMergeStoreRWTest, TestFastScanComplexWithModeConversion) try { const size_t num_write_rows = 128; @@ -1053,7 +1053,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; switch (mode) @@ -1133,7 +1133,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ false, + /* is_fast_scan= */ false, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -1143,7 +1143,7 @@ try } CATCH -TEST_P(DeltaMergeStoreRWTest, TestFastModeForCleanRead) +TEST_P(DeltaMergeStoreRWTest, TestFastScanForCleanRead) try { const size_t num_rows_write = 128; @@ -1185,7 +1185,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_COLS_UR( in, @@ -1229,7 +1229,7 @@ try EMPTY_FILTER, TRACING_NAME, /* keep_order= */ false, - /* is_fast_mode= */ true, + /* is_fast_scan= */ true, /* expected_block_size= */ 1024)[0]; ASSERT_INPUTSTREAM_NROWS(in, num_rows_write - num_deleted_rows); } diff --git a/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp index ba78ea6c606..bf241efa0b4 100644 --- a/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp @@ -192,7 +192,7 @@ void DTWorkload::read(const ColumnDefines & columns, int stream_count, T func) auto filter = EMPTY_FILTER; int excepted_block_size = 1024; uint64_t read_ts = ts_gen->get(); - auto streams = store->read(*context, context->getSettingsRef(), columns, ranges, stream_count, read_ts, filter, "DTWorkload", false, opts->is_fast_mode, excepted_block_size); + auto streams = store->read(*context, context->getSettingsRef(), columns, ranges, stream_count, read_ts, filter, "DTWorkload", false, opts->is_fast_scan, excepted_block_size); std::vector threads; threads.reserve(streams.size()); for (auto & stream : streams) diff --git a/dbms/src/Storages/DeltaMerge/workload/Options.cpp b/dbms/src/Storages/DeltaMerge/workload/Options.cpp index f10f008ced5..40fd3c594a3 100644 --- a/dbms/src/Storages/DeltaMerge/workload/Options.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/Options.cpp @@ -48,7 +48,7 @@ std::string WorkloadOptions::toString(std::string seperator) const fmt::format("bg_thread_count {}{}", bg_thread_count, seperator) + // fmt::format("table_id {}{}", table_id, seperator) + // fmt::format("table_name {}{}", table_name, seperator) + // - fmt::format("is_fast_mode {}{}", is_fast_mode, seperator) + // + fmt::format("is_fast_scan {}{}", is_fast_scan, seperator) + // fmt::format("enable_read_thread {}{}", enable_read_thread, seperator); } @@ -96,11 +96,8 @@ std::pair WorkloadOptions::parseOptions(int argc, char * argv // ("table_name", value()->default_value(""), "") // ("table_id", value()->default_value(-1), "") // - // - ("is_fast_mode", value()->default_value(false), "default is false, means normal mode. When we in fast mode, we should set verification as false") // - // - ("enable_read_thread", value()->default_value(true), "") // - ; + ("is_fast_scan", value()->default_value(false), "default is false, means normal mode. When we in fast mode, we should set verification as false") // + ("enable_read_thread", value()->default_value(true), ""); boost::program_options::variables_map vm; boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), vm); @@ -180,11 +177,11 @@ std::pair WorkloadOptions::parseOptions(int argc, char * argv table_id = vm["table_id"].as(); table_name = vm["table_name"].as(); - is_fast_mode = vm["is_fast_mode"].as(); + is_fast_scan = vm["is_fast_scan"].as(); - if (is_fast_mode && verification) + if (is_fast_scan && verification) { - return {false, fmt::format("When in_fast_mode, we should set verification as false")}; + return {false, fmt::format("When in_fast_scan, we should set verification as false")}; } enable_read_thread = vm["enable_read_thread"].as(); diff --git a/dbms/src/Storages/DeltaMerge/workload/Options.h b/dbms/src/Storages/DeltaMerge/workload/Options.h index 2a0860b5d2b..61844a53211 100644 --- a/dbms/src/Storages/DeltaMerge/workload/Options.h +++ b/dbms/src/Storages/DeltaMerge/workload/Options.h @@ -62,7 +62,7 @@ struct WorkloadOptions int64_t table_id; std::string table_name; - bool is_fast_mode; + bool is_fast_scan; bool enable_read_thread; diff --git a/dbms/src/Storages/Page/V3/LogFile/LogWriter.cpp b/dbms/src/Storages/Page/V3/LogFile/LogWriter.cpp index 2954172cb21..d7ef55e6e4b 100644 --- a/dbms/src/Storages/Page/V3/LogFile/LogWriter.cpp +++ b/dbms/src/Storages/Page/V3/LogFile/LogWriter.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -46,6 +47,7 @@ LogWriter::LogWriter( /*create_new_encryption_info_*/ true); buffer = static_cast(alloc(buffer_size)); + RUNTIME_CHECK_MSG(buffer != nullptr, "LogWriter cannot allocate buffer, size={}", buffer_size); write_buffer = WriteBuffer(buffer, buffer_size); } @@ -67,8 +69,13 @@ size_t LogWriter::writtenBytes() const return written_bytes; } -void LogWriter::flush(const WriteLimiterPtr & write_limiter, const bool background) +void LogWriter::flush(const WriteLimiterPtr & write_limiter, bool background) { + if (write_buffer.offset() == 0) + { + return; + } + PageUtil::writeFile(log_file, written_bytes, write_buffer.buffer().begin(), @@ -89,43 +96,35 @@ void LogWriter::close() log_file->close(); } -void LogWriter::addRecord(ReadBuffer & payload, const size_t payload_size, const WriteLimiterPtr & write_limiter) +void LogWriter::addRecord(ReadBuffer & payload, const size_t payload_size, const WriteLimiterPtr & write_limiter, bool background) { // Header size varies depending on whether we are recycling or not. - const int header_size = recycle_log_files ? Format::RECYCLABLE_HEADER_SIZE : Format::HEADER_SIZE; + const UInt32 header_size = recycle_log_files ? Format::RECYCLABLE_HEADER_SIZE : Format::HEADER_SIZE; // Fragment the record if necessary and emit it. Note that if payload is empty, // we still want to iterate once to emit a single zero-length record. bool begin = true; size_t payload_left = payload_size; - - size_t head_sizes = ((payload_size / Format::BLOCK_SIZE) + 1) * Format::RECYCLABLE_HEADER_SIZE; - if (payload_size + head_sizes >= buffer_size) + // Padding current block if needed + block_offset = block_offset % Format::BLOCK_SIZE; // 0 <= block_offset < Format::BLOCK_SIZE + size_t leftover = Format::BLOCK_SIZE - block_offset; + assert(leftover > 0); + if (leftover < header_size) { - size_t new_buff_size = payload_size + ((head_sizes / Format::BLOCK_SIZE) + 1) * Format::BLOCK_SIZE; - - buffer = static_cast(realloc(buffer, buffer_size, new_buff_size)); - buffer_size = new_buff_size; - resetBuffer(); + // Fill the trailer with all zero + static constexpr char MAX_ZERO_HEADER[Format::RECYCLABLE_HEADER_SIZE]{'\x00'}; + if (unlikely(buffer_size - write_buffer.offset() < leftover)) + { + flush(write_limiter, background); + } + writeString(MAX_ZERO_HEADER, leftover, write_buffer); + block_offset = 0; } - do { - const Int64 leftover = Format::BLOCK_SIZE - block_offset; - assert(leftover >= 0); - if (leftover < header_size) - { - // Switch to a new block - if (leftover > 0) - { - // Fill the trailer with all zero - static constexpr char MAX_ZERO_HEADER[Format::RECYCLABLE_HEADER_SIZE]{'\x00'}; - writeString(MAX_ZERO_HEADER, leftover, write_buffer); - } - block_offset = 0; - } + block_offset = block_offset % Format::BLOCK_SIZE; // Invariant: we never leave < header_size bytes in a block. - assert(static_cast(Format::BLOCK_SIZE - block_offset) >= header_size); + assert(Format::BLOCK_SIZE - block_offset >= header_size); const size_t avail_payload_size = Format::BLOCK_SIZE - block_offset - header_size; const size_t fragment_length = (payload_left < avail_payload_size) ? payload_left : avail_payload_size; @@ -139,7 +138,21 @@ void LogWriter::addRecord(ReadBuffer & payload, const size_t payload_size, const type = recycle_log_files ? Format::RecordType::RecyclableLastType : Format::RecordType::LastType; else type = recycle_log_files ? Format::RecordType::RecyclableMiddleType : Format::RecordType::MiddleType; - emitPhysicalRecord(type, payload, fragment_length); + // Check available space in write_buffer before writing + if (buffer_size - write_buffer.offset() < fragment_length + header_size) + { + flush(write_limiter, background); + } + try + { + emitPhysicalRecord(type, payload, fragment_length); + } + catch (...) + { + auto message = getCurrentExceptionMessage(true); + LOG_FMT_FATAL(&Poco::Logger::get("LogWriter"), "Write physical record failed with message: {}", message); + std::terminate(); + } payload.ignore(fragment_length); payload_left -= fragment_length; begin = false; @@ -147,7 +160,7 @@ void LogWriter::addRecord(ReadBuffer & payload, const size_t payload_size, const if (!manual_flush) { - flush(write_limiter, /* background */ false); + flush(write_limiter, background); } } diff --git a/dbms/src/Storages/Page/V3/LogFile/LogWriter.h b/dbms/src/Storages/Page/V3/LogFile/LogWriter.h index 6cd8f3b46b4..8b1b791b67c 100644 --- a/dbms/src/Storages/Page/V3/LogFile/LogWriter.h +++ b/dbms/src/Storages/Page/V3/LogFile/LogWriter.h @@ -64,11 +64,11 @@ namespace PS::V3 * Recyclable record format: * * +--------------+-----------+-----------+----------------+--- ... ---+ - * |CheckSum (8B) | Size (2B) | Type (1B) | Log number (4B)| Payload | + * |CheckSum (8B) | Size (2B) | Type (1B) | Log number (8B)| Payload | * +--------------+-----------+-----------+----------------+--- ... ---+ * * Same as above, with the addition of - * Log number = 32bit log file number, so that we can distinguish between + * Log number = 64bit log file number, so that we can distinguish between * records written by the most recent log writer vs a previous one. */ class LogWriter final : private Allocator @@ -85,9 +85,9 @@ class LogWriter final : private Allocator ~LogWriter(); - void addRecord(ReadBuffer & payload, size_t payload_size, const WriteLimiterPtr & write_limiter = nullptr); + void addRecord(ReadBuffer & payload, size_t payload_size, const WriteLimiterPtr & write_limiter = nullptr, bool background = false); - void flush(const WriteLimiterPtr & write_limiter = nullptr, const bool background = false); + void flush(const WriteLimiterPtr & write_limiter = nullptr, bool background = false); void close(); @@ -119,7 +119,7 @@ class LogWriter final : private Allocator size_t written_bytes = 0; char * buffer; - size_t buffer_size = Format::BLOCK_SIZE; + const size_t buffer_size = Format::BLOCK_SIZE; WriteBuffer write_buffer; }; } // namespace PS::V3 diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index 67360db8747..5dbda3b1927 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -1170,8 +1170,7 @@ void PageDirectory::apply(PageEntriesEdit && edit, const WriteLimiterPtr & write { // put the new created holder into `external_ids` *holder = r.page_id; - std::lock_guard guard(external_ids_mutex); - external_ids.emplace_back(std::weak_ptr(holder)); + external_ids_by_ns.addExternalId(holder); } break; } @@ -1235,26 +1234,6 @@ void PageDirectory::gcApply(PageEntriesEdit && migrated_edit, const WriteLimiter LOG_FMT_INFO(log, "GC apply done. [edit size={}]", migrated_edit.size()); } -std::set PageDirectory::getAliveExternalIds(NamespaceId ns_id) const -{ - std::set valid_external_ids; - { - std::lock_guard guard(external_ids_mutex); - for (auto iter = external_ids.begin(); iter != external_ids.end(); /*empty*/) - { - if (auto holder = iter->lock(); holder == nullptr) - iter = external_ids.erase(iter); - else - { - if (holder->high == ns_id) - valid_external_ids.emplace(holder->low); - ++iter; - } - } - } - return valid_external_ids; -} - std::pair, PageSize> PageDirectory::getEntriesByBlobIds(const std::vector & blob_ids) const { diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index 6042238e598..065d5a86f34 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -20,9 +20,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -348,7 +350,20 @@ class PageDirectory // When dump snapshot, we need to keep the last valid entry. Check out `tryDumpSnapshot` for the reason. PageEntriesV3 gcInMemEntries(bool return_removed_entries = true, bool keep_last_valid_var_entry = false); - std::set getAliveExternalIds(NamespaceId ns_id) const; + // Get the external id that is not deleted or being ref by another id by + // `ns_id`. + std::set getAliveExternalIds(NamespaceId ns_id) const + { + return external_ids_by_ns.getAliveIds(ns_id); + } + + // After table dropped, the `getAliveIds` with specified + // `ns_id` will not be cleaned. We need this method to + // cleanup all external id ptrs. + void unregisterNamespace(NamespaceId ns_id) + { + external_ids_by_ns.unregisterNamespace(ns_id); + } PageEntriesEdit dumpSnapshotToEdit(PageDirectorySnapshotPtr snap = nullptr); @@ -391,8 +406,7 @@ class PageDirectory mutable std::mutex snapshots_mutex; mutable std::list> snapshots; - mutable std::mutex external_ids_mutex; - mutable std::list> external_ids; + mutable ExternalIdsByNamespace external_ids_by_ns; WALStorePtr wal; const UInt64 max_persisted_log_files; diff --git a/dbms/src/Storages/Page/V3/PageDirectory/ExternalIdsByNamespace.cpp b/dbms/src/Storages/Page/V3/PageDirectory/ExternalIdsByNamespace.cpp new file mode 100644 index 00000000000..217b8896f7e --- /dev/null +++ b/dbms/src/Storages/Page/V3/PageDirectory/ExternalIdsByNamespace.cpp @@ -0,0 +1,78 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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::PS::V3 +{ +void ExternalIdsByNamespace::addExternalIdUnlock(const std::shared_ptr & external_id) +{ + const NamespaceId & ns_id = external_id->high; + // create a new ExternalIds if the ns_id is not exists, else return + // the existing one. + auto [ns_iter, new_inserted] = ids_by_ns.try_emplace(ns_id, ExternalIds{}); + ns_iter->second.emplace_back(std::weak_ptr(external_id)); +} + +void ExternalIdsByNamespace::addExternalId(const std::shared_ptr & external_id) +{ + std::unique_lock map_guard(mu); + addExternalIdUnlock(external_id); +} + +std::set ExternalIdsByNamespace::getAliveIds(NamespaceId ns_id) const +{ + // Now we assume a lock among all NamespaceIds is good enough. + std::unique_lock map_guard(mu); + + std::set valid_external_ids; + auto ns_iter = ids_by_ns.find(ns_id); + if (ns_iter == ids_by_ns.end()) + return valid_external_ids; + + // Only scan the given `ns_id` + auto & external_ids = ns_iter->second; + for (auto iter = external_ids.begin(); iter != external_ids.end(); /*empty*/) + { + if (auto holder = iter->lock(); holder == nullptr) + { + // the external id has been removed from `PageDirectory`, + // cleanup the invalid weak_ptr + iter = external_ids.erase(iter); + continue; + } + else + { + valid_external_ids.emplace(holder->low); + ++iter; + } + } + // No valid external pages in this `ns_id` + if (valid_external_ids.empty()) + { + valid_external_ids.erase(ns_id); + } + return valid_external_ids; +} + +void ExternalIdsByNamespace::unregisterNamespace(NamespaceId ns_id) +{ + std::unique_lock map_guard(mu); + // free all weak_ptrs of this namespace + ids_by_ns.erase(ns_id); +} +} // namespace DB::PS::V3 diff --git a/dbms/src/Storages/Page/V3/PageDirectory/ExternalIdsByNamespace.h b/dbms/src/Storages/Page/V3/PageDirectory/ExternalIdsByNamespace.h new file mode 100644 index 00000000000..26ff9c109f4 --- /dev/null +++ b/dbms/src/Storages/Page/V3/PageDirectory/ExternalIdsByNamespace.h @@ -0,0 +1,56 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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::PS::V3 +{ + +// A thread-safe class to manage external ids. +// Manage all external ids by NamespaceId. +class ExternalIdsByNamespace +{ +public: + ExternalIdsByNamespace() = default; + + // Add a external ids + void addExternalId(const std::shared_ptr & external_id); + // non thread-safe version, only for restore + void addExternalIdUnlock(const std::shared_ptr & external_id); + + // Get all alive external ids of given `ns_id` + // Will also cleanup the invalid external ids. + std::set getAliveIds(NamespaceId ns_id) const; + + // After table dropped, the `getAliveIds` with specified + // `ns_id` will not be cleaned. We need this method to + // cleanup all external id ptrs. + void unregisterNamespace(NamespaceId ns_id); + + DISALLOW_COPY_AND_MOVE(ExternalIdsByNamespace); + +private: + mutable std::mutex mu; + // Only store weak_ptrs. The weak_ptrs will be invalid after the external id + // in PageDirectory get removed. + using ExternalIds = std::list>; + using NamespaceMap = std::unordered_map; + mutable NamespaceMap ids_by_ns; +}; +} // namespace DB::PS::V3 diff --git a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp index aadef4d12a9..8bb75af72cb 100644 --- a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp @@ -149,7 +149,7 @@ void PageDirectoryFactory::applyRecord( if (holder) { *holder = r.page_id; - dir->external_ids.emplace_back(std::weak_ptr(holder)); + dir->external_ids_by_ns.addExternalIdUnlock(holder); } break; } @@ -162,7 +162,7 @@ void PageDirectoryFactory::applyRecord( if (holder) { *holder = r.page_id; - dir->external_ids.emplace_back(std::weak_ptr(holder)); + dir->external_ids_by_ns.addExternalIdUnlock(holder); } break; } diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp index 1bd79f54204..dba1fef7566 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp @@ -12,14 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include #include #include +#include #include #include #include #include #include #include +#include namespace DB { @@ -269,6 +273,51 @@ void PageStorageImpl::traverseImpl(const std::function"; + case GCStageType::OnlyInMem: + return " without full gc"; + case GCStageType::FullGCNothingMoved: + return " without moving any entry"; + case GCStageType::FullGC: + return ""; + } + }(); + const auto get_external_msg = [this]() -> String { + if (clean_external_page_ms == 0) + return String(""); + static constexpr double SCALE_NS_TO_MS = 1'000'000.0; + return fmt::format(" [external_callbacks={}] [external_gc={}ms] [scanner={:.2f}ms] [get_alive={:.2f}ms] [remover={:.2f}ms]", + num_external_callbacks, + clean_external_page_ms, + external_page_scan_ns / SCALE_NS_TO_MS, + external_page_get_alive_ns / SCALE_NS_TO_MS, + external_page_remove_ns / SCALE_NS_TO_MS); + }; + return fmt::format("GC finished{}." + " [total time={}ms]" + " [dump snapshots={}ms] [gc in mem entries={}ms]" + " [blobstore remove entries={}ms] [blobstore get status={}ms]" + " [get gc entries={}ms] [blobstore full gc={}ms]" + " [gc apply={}ms]" + "{}", // a placeholder for external page gc at last + stage_suffix, + total_cost_ms, + dump_snapshots_ms, + gc_in_mem_entries_ms, + blobstore_remove_entries_ms, + blobstore_get_gc_stats_ms, + full_gc_get_entries_ms, + full_gc_blobstore_copy_ms, + full_gc_apply_ms, + get_external_msg()); +} + bool PageStorageImpl::gcImpl(bool /*not_skip*/, const WriteLimiterPtr & write_limiter, const ReadLimiterPtr & read_limiter) { // If another thread is running gc, just return; @@ -276,6 +325,43 @@ bool PageStorageImpl::gcImpl(bool /*not_skip*/, const WriteLimiterPtr & write_li if (!gc_is_running.compare_exchange_strong(v, true)) return false; + const GCTimeStatistics statistics = doGC(write_limiter, read_limiter); + assert(statistics.stage != GCStageType::Unknown); // `doGC` must set the stage + LOG_DEBUG(log, statistics.toLogging()); + + return statistics.executeNextImmediately(); +} + +// Remove external pages for all tables +// TODO: `clean_external_page` for all tables may slow down the whole gc process when there are lots of table. +void PageStorageImpl::cleanExternalPage(Stopwatch & gc_watch, GCTimeStatistics & statistics) +{ + // TODO: `callbacks_mutex` is being held during the whole `cleanExternalPage`, meaning gc will block + // creating/dropping table, need to refine it later. + std::scoped_lock lock{callbacks_mutex}; + statistics.num_external_callbacks = callbacks_container.size(); + if (!callbacks_container.empty()) + { + Stopwatch external_watch; + for (const auto & [ns_id, callbacks] : callbacks_container) + { + // Note that we must call `scanner` before `getAliveExternalIds` + // Or some committed external ids is not included and we may + // remove the external page by accident with `remover`. + const auto pending_external_pages = callbacks.scanner(); + statistics.external_page_scan_ns += external_watch.elapsedFromLastTime(); + const auto alive_external_ids = page_directory->getAliveExternalIds(ns_id); + statistics.external_page_get_alive_ns += external_watch.elapsedFromLastTime(); + callbacks.remover(pending_external_pages, alive_external_ids); + statistics.external_page_remove_ns += external_watch.elapsedFromLastTime(); + } + } + + statistics.clean_external_page_ms = gc_watch.elapsedMillisecondsFromLastTime(); +} + +PageStorageImpl::GCTimeStatistics PageStorageImpl::doGC(const WriteLimiterPtr & write_limiter, const ReadLimiterPtr & read_limiter) +{ Stopwatch gc_watch; SCOPE_EXIT({ GET_METRIC(tiflash_storage_page_gc_count, type_v3).Increment(); @@ -284,18 +370,7 @@ bool PageStorageImpl::gcImpl(bool /*not_skip*/, const WriteLimiterPtr & write_li gc_is_running.compare_exchange_strong(is_running, false); }); - auto clean_external_page = [this]() { - std::scoped_lock lock{callbacks_mutex}; - if (!callbacks_container.empty()) - { - for (const auto & [ns_id, callbacks] : callbacks_container) - { - auto pending_external_pages = callbacks.scanner(); - auto alive_external_ids = getAliveExternalPageIds(ns_id); - callbacks.remover(pending_external_pages, alive_external_ids); - } - } - }; + GCTimeStatistics statistics; // 1. Do the MVCC gc, clean up expired snapshot. // And get the expired entries. @@ -303,71 +378,50 @@ bool PageStorageImpl::gcImpl(bool /*not_skip*/, const WriteLimiterPtr & write_li { GET_METRIC(tiflash_storage_page_gc_count, type_v3_mvcc_dumped).Increment(); } - const auto dump_snapshots_ms = gc_watch.elapsedMillisecondsFromLastTime(); + statistics.dump_snapshots_ms = gc_watch.elapsedMillisecondsFromLastTime(); const auto & del_entries = page_directory->gcInMemEntries(); - const auto gc_in_mem_entries_ms = gc_watch.elapsedMillisecondsFromLastTime(); + statistics.gc_in_mem_entries_ms = gc_watch.elapsedMillisecondsFromLastTime(); // 2. Remove the expired entries in BlobStore. // It won't delete the data on the disk. // It will only update the SpaceMap which in memory. blob_store.remove(del_entries); - const auto blobstore_remove_entries_ms = gc_watch.elapsedMillisecondsFromLastTime(); + statistics.blobstore_remove_entries_ms = gc_watch.elapsedMillisecondsFromLastTime(); - // 3. Analyze the status of each Blob in order to obtain the Blobs that need to do `heavy GC`. - // Blobs that do not need to do heavy GC will also do ftruncate to reduce space enlargement. - const auto & blob_need_gc = blob_store.getGCStats(); - const auto blobstore_get_gc_stats_ms = gc_watch.elapsedMillisecondsFromLastTime(); - if (blob_need_gc.empty()) - { - LOG_FMT_INFO(log, "GC finished without any blob need full gc. [total time(ms)={}]" - " [dump snapshots(ms)={}] [gc in mem entries(ms)={}]" - " [blobstore remove entries(ms)={}] [blobstore get status(ms)={}]", - gc_watch.elapsedMilliseconds(), - dump_snapshots_ms, - gc_in_mem_entries_ms, - blobstore_remove_entries_ms, - blobstore_get_gc_stats_ms); - clean_external_page(); - return false; - } - else + // 3. Analyze the status of each Blob in order to obtain the Blobs that need to do `full GC`. + // Blobs that do not need to do full GC will also do ftruncate to reduce space amplification. + const auto & blob_ids_need_gc = blob_store.getGCStats(); + statistics.blobstore_get_gc_stats_ms = gc_watch.elapsedMillisecondsFromLastTime(); + if (blob_ids_need_gc.empty()) { - GET_METRIC(tiflash_storage_page_gc_count, type_v3_bs_full_gc).Increment(blob_need_gc.size()); + cleanExternalPage(gc_watch, statistics); + statistics.stage = GCStageType::OnlyInMem; + statistics.total_cost_ms = gc_watch.elapsedMilliseconds(); + return statistics; } // Execute full gc + GET_METRIC(tiflash_storage_page_gc_count, type_v3_bs_full_gc).Increment(blob_ids_need_gc.size()); // 4. Filter out entries in MVCC by BlobId. // We also need to filter the version of the entry. // So that the `gc_apply` can proceed smoothly. - auto [blob_gc_info, total_page_size] = page_directory->getEntriesByBlobIds(blob_need_gc); - const auto gc_get_entries_ms = gc_watch.elapsedMillisecondsFromLastTime(); + auto [blob_gc_info, total_page_size] = page_directory->getEntriesByBlobIds(blob_ids_need_gc); + statistics.full_gc_get_entries_ms = gc_watch.elapsedMillisecondsFromLastTime(); if (blob_gc_info.empty()) { - LOG_FMT_INFO(log, "GC finished without any entry need be moved. [total time(ms)={}]" - " [dump snapshots(ms)={}] [in mem entries(ms)={}]" - " [blobstore remove entries(ms)={}] [blobstore get status(ms)={}]" - " [get entries(ms)={}]", - gc_watch.elapsedMilliseconds(), - dump_snapshots_ms, - gc_in_mem_entries_ms, - blobstore_remove_entries_ms, - blobstore_get_gc_stats_ms, - gc_get_entries_ms); - - clean_external_page(); - return false; + cleanExternalPage(gc_watch, statistics); + statistics.stage = GCStageType::FullGCNothingMoved; + statistics.total_cost_ms = gc_watch.elapsedMilliseconds(); + return statistics; } // 5. Do the BlobStore GC // After BlobStore GC, these entries will be migrated to a new blob. // Then we should notify MVCC apply the change. PageEntriesEdit gc_edit = blob_store.gc(blob_gc_info, total_page_size, write_limiter, read_limiter); - const auto blobstore_full_gc_ms = gc_watch.elapsedMillisecondsFromLastTime(); - if (gc_edit.empty()) - { - throw Exception("Something wrong after BlobStore GC.", ErrorCodes::LOGICAL_ERROR); - } + statistics.full_gc_blobstore_copy_ms = gc_watch.elapsedMillisecondsFromLastTime(); + RUNTIME_CHECK_MSG(!gc_edit.empty(), "Something wrong after BlobStore GC"); // 6. MVCC gc apply // MVCC will apply the migrated entries. @@ -377,24 +431,12 @@ bool PageStorageImpl::gcImpl(bool /*not_skip*/, const WriteLimiterPtr & write_li // will be remained as "read-only" files while entries in them are useless in actual. // Those BlobFiles should be cleaned during next restore. page_directory->gcApply(std::move(gc_edit), write_limiter); - const auto gc_apply_ms = gc_watch.elapsedMillisecondsFromLastTime(); - LOG_FMT_INFO(log, "GC finished. [total time(ms)={}]" - " [dump snapshots(ms)={}] [gc in mem entries(ms)={}]" - " [blobstore remove entries(ms)={}] [blobstore get status(ms)={}]" - " [get gc entries(ms)={}] [blobstore full gc(ms)={}]" - " [gc apply(ms)={}]", - gc_watch.elapsedMilliseconds(), - dump_snapshots_ms, - gc_in_mem_entries_ms, - blobstore_remove_entries_ms, - blobstore_get_gc_stats_ms, - gc_get_entries_ms, - blobstore_full_gc_ms, - gc_apply_ms); - - clean_external_page(); - - return true; + statistics.full_gc_apply_ms = gc_watch.elapsedMillisecondsFromLastTime(); + + cleanExternalPage(gc_watch, statistics); + statistics.stage = GCStageType::FullGC; + statistics.total_cost_ms = gc_watch.elapsedMilliseconds(); + return statistics; } void PageStorageImpl::registerExternalPagesCallbacks(const ExternalPageCallbacks & callbacks) @@ -409,8 +451,12 @@ void PageStorageImpl::registerExternalPagesCallbacks(const ExternalPageCallbacks void PageStorageImpl::unregisterExternalPagesCallbacks(NamespaceId ns_id) { - std::scoped_lock lock{callbacks_mutex}; - callbacks_container.erase(ns_id); + { + std::scoped_lock lock{callbacks_mutex}; + callbacks_container.erase(ns_id); + } + // clean all external ids ptrs + page_directory->unregisterNamespace(ns_id); } const String PageStorageImpl::manifests_file_name = "manifests"; diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.h b/dbms/src/Storages/Page/V3/PageStorageImpl.h index a1165d0e9b2..9bce2e5dde8 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.h +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.h @@ -108,12 +108,14 @@ class PageStorageImpl : public DB::PageStorage #ifndef NDEBUG // Just for tests, refactor them out later + // clang-format off DB::PageStorage::SnapshotPtr getSnapshot() { return getSnapshot(""); } DB::PageEntry getEntry(PageId page_id) { return getEntryImpl(TEST_NAMESPACE_ID, page_id, nullptr); } DB::Page read(PageId page_id) { return readImpl(TEST_NAMESPACE_ID, page_id, nullptr, nullptr, true); } PageMap read(const PageIds & page_ids) { return readImpl(TEST_NAMESPACE_ID, page_ids, nullptr, nullptr, true); } PageIds read(const PageIds & page_ids, const PageHandler & handler) { return readImpl(TEST_NAMESPACE_ID, page_ids, handler, nullptr, nullptr, true); } PageMap read(const std::vector & page_fields) { return readImpl(TEST_NAMESPACE_ID, page_fields, nullptr, nullptr, true); } + // clang-format on #endif friend class PageDirectoryFactory; @@ -121,6 +123,44 @@ class PageStorageImpl : public DB::PageStorage #ifndef DBMS_PUBLIC_GTEST private: #endif + + enum class GCStageType + { + Unknown, + OnlyInMem, + FullGCNothingMoved, + FullGC, + }; + struct GCTimeStatistics + { + GCStageType stage = GCStageType::Unknown; + bool executeNextImmediately() const { return stage == GCStageType::FullGC; }; + + UInt64 total_cost_ms = 0; + + UInt64 dump_snapshots_ms = 0; + UInt64 gc_in_mem_entries_ms = 0; + UInt64 blobstore_remove_entries_ms = 0; + UInt64 blobstore_get_gc_stats_ms = 0; + // Full GC + UInt64 full_gc_get_entries_ms = 0; + UInt64 full_gc_blobstore_copy_ms = 0; + UInt64 full_gc_apply_ms = 0; + + // GC external page + UInt64 clean_external_page_ms = 0; + UInt64 num_external_callbacks = 0; + // ms is usually too big for these operation, store by ns (10^-9) + UInt64 external_page_scan_ns = 0; + UInt64 external_page_get_alive_ns = 0; + UInt64 external_page_remove_ns = 0; + + String toLogging() const; + }; + + GCTimeStatistics doGC(const WriteLimiterPtr & write_limiter, const ReadLimiterPtr & read_limiter); + void cleanExternalPage(Stopwatch & gc_watch, GCTimeStatistics & statistics); + LoggerPtr log; PageDirectoryPtr page_directory; diff --git a/dbms/src/Storages/Page/V3/WALStore.cpp b/dbms/src/Storages/Page/V3/WALStore.cpp index c7f11ee8b3c..8afc52c6dca 100644 --- a/dbms/src/Storages/Page/V3/WALStore.cpp +++ b/dbms/src/Storages/Page/V3/WALStore.cpp @@ -201,7 +201,7 @@ bool WALStore::saveSnapshot(FilesSnapshot && files_snap, PageEntriesEdit && dire const String serialized = ser::serializeTo(directory_snap); ReadBufferFromString payload(serialized); - compact_log->addRecord(payload, serialized.size()); + compact_log->addRecord(payload, serialized.size(), write_limiter, /*background*/ true); compact_log->flush(write_limiter, /*background*/ true); compact_log.reset(); // close fd explicitly before renaming file. diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp index 134e3c51350..5fce5797c8d 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp @@ -14,12 +14,14 @@ #include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -34,6 +36,7 @@ #include #include +#include #include #include #include @@ -44,6 +47,43 @@ namespace DB { namespace PS::V3::tests { +TEST(ExternalIdsByNamespace, Simple) +{ + NamespaceId ns_id = 100; + ExternalIdsByNamespace external_ids_by_ns; + + std::atomic who(0); + + std::shared_ptr holder = std::make_shared(buildV3Id(ns_id, 50)); + + auto th_insert = std::async([&]() { + external_ids_by_ns.addExternalId(holder); + + Int32 expect = 0; + who.compare_exchange_strong(expect, 1); + }); + auto th_get_alive = std::async([&]() { + external_ids_by_ns.getAliveIds(ns_id); + Int32 expect = 0; + who.compare_exchange_strong(expect, 2); + }); + th_get_alive.wait(); + th_insert.wait(); + + { + auto ids = external_ids_by_ns.getAliveIds(ns_id); + LOG_DEBUG(&Poco::Logger::root(), "{} end first, size={}", who.load(), ids.size()); + ASSERT_EQ(ids.size(), 1); + ASSERT_EQ(*ids.begin(), 50); + } + + { + external_ids_by_ns.unregisterNamespace(ns_id); + auto ids = external_ids_by_ns.getAliveIds(ns_id); + ASSERT_EQ(ids.size(), 0); + } +} + class PageDirectoryTest : public DB::base::TiFlashStorageTestBasic { public: diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp index e019626197e..0ccdd1d9b2c 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -1157,16 +1157,37 @@ try size_t times_remover_called = 0; + enum + { + STAGE_SNAP_KEEP = 1, + STAGE_SNAP_RELEASED = 2, + } test_stage; + test_stage = STAGE_SNAP_KEEP; + ExternalPageCallbacks callbacks; callbacks.scanner = []() -> ExternalPageCallbacks::PathAndIdsVec { return {}; }; - callbacks.remover = [×_remover_called](const ExternalPageCallbacks::PathAndIdsVec &, const std::set & living_page_ids) -> void { + callbacks.remover = [×_remover_called, &test_stage](const ExternalPageCallbacks::PathAndIdsVec &, const std::set & living_page_ids) -> void { times_remover_called += 1; - // 0, 1024 are still alive - EXPECT_EQ(living_page_ids.size(), 2); - EXPECT_GT(living_page_ids.count(0), 0); - EXPECT_GT(living_page_ids.count(1024), 0); + switch (test_stage) + { + case STAGE_SNAP_KEEP: + { + // 0, 1024 are still alive + EXPECT_EQ(living_page_ids.size(), 2); + EXPECT_GT(living_page_ids.count(0), 0); + EXPECT_GT(living_page_ids.count(1024), 0); + break; + } + case STAGE_SNAP_RELEASED: + { + /// After `snapshot` released, 1024 should be removed from `living` + EXPECT_EQ(living_page_ids.size(), 1); + EXPECT_GT(living_page_ids.count(0), 0); + break; + } + } }; callbacks.ns_id = TEST_NAMESPACE_ID; page_storage->registerExternalPagesCallbacks(callbacks); @@ -1208,13 +1229,7 @@ try /// After `snapshot` released, 1024 should be removed from `living` snapshot.reset(); - callbacks.remover = [×_remover_called](const ExternalPageCallbacks::PathAndIdsVec &, const std::set & living_page_ids) -> void { - times_remover_called += 1; - EXPECT_EQ(living_page_ids.size(), 1); - EXPECT_GT(living_page_ids.count(0), 0); - }; - page_storage->unregisterExternalPagesCallbacks(callbacks.ns_id); - page_storage->registerExternalPagesCallbacks(callbacks); + test_stage = STAGE_SNAP_RELEASED; { SCOPED_TRACE("gc with snapshot released"); page_storage->gc(); diff --git a/dbms/src/Storages/Page/V3/tests/gtest_wal_log.cpp b/dbms/src/Storages/Page/V3/tests/gtest_wal_log.cpp index c4d3e2f7ab7..a962c5d27e0 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_wal_log.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_wal_log.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -275,6 +276,17 @@ TEST_P(LogFileRWTest, BlockBoundary) ASSERT_EQ("EOF", read()); } +TEST_P(LogFileRWTest, BlockBoundaryEmpty) +{ + const auto big_str = repeatedString("A", PS::V3::Format::BLOCK_SIZE - Format::HEADER_SIZE - 4); + write(big_str); + write(""); + resetReader(); + ASSERT_EQ(big_str, read()); + ASSERT_EQ("", read()); + ASSERT_EQ("EOF", read()); +} + TEST_P(LogFileRWTest, ManyBlocks) { const size_t num_blocks_test = 100000; @@ -303,6 +315,15 @@ TEST_P(LogFileRWTest, Fragmentation) ASSERT_EQ("EOF", read()); } +// This test may take a lot of time +TEST_P(LogFileRWTest, DISABLED__DifferentPayloadSize) +{ + for (size_t i = 0; i < 40000; i += 1) + { + write(repeatedString("a", i)); + } +} + TEST_P(LogFileRWTest, MarginalTrailer) { // Make a trailer that is exactly the same length as an empty record. @@ -789,4 +810,58 @@ INSTANTIATE_TEST_CASE_P( return fmt::format("{}_{}", recycle_log, allow_retry_read); }); +TEST(LogFileRWTest2, ManuallyFlush) +{ + auto provider = TiFlashTestEnv::getContext().getFileProvider(); + auto path = TiFlashTestEnv::getTemporaryPath("LogFileRWTest2"); + DB::tests::TiFlashTestEnv::tryRemovePath(path); + + Poco::File file(path); + if (!file.exists()) + { + file.createDirectories(); + } + + auto file_name = path + "/log_0"; + auto payload = repeatedString("medium", 50000); + Format::LogNumberType log_num = 30; + + auto writer = std::make_unique(file_name, provider, log_num, /* recycle_log */ true, /* manual_flush */ true); + { + ReadBufferFromString buff(payload); + ASSERT_NO_THROW(writer->addRecord(buff, payload.size())); + } + { + ReadBufferFromString buff(payload); + ASSERT_NO_THROW(writer->addRecord(buff, payload.size())); + } + writer->flush(); + + auto read_buf = createReadBufferFromFileBaseByFileProvider( + provider, + file_name, + EncryptionPath{file_name, ""}, + /*estimated_size*/ Format::BLOCK_SIZE, + /*aio_threshold*/ 0, + /*read_limiter*/ nullptr, + /*buffer_size*/ Format::BLOCK_SIZE // Must be `Format::BLOCK_SIZE` + ); + + DB::PS::V3::ReportCollector reporter; + auto reader = std::make_unique(std::move(read_buf), + &reporter, + /* verify_checksum */ true, + log_num, + WALRecoveryMode::PointInTimeRecovery); + { + auto [ok, scratch] = reader->readRecord(); + ASSERT_TRUE(ok); + ASSERT_EQ(scratch, payload); + } + { + auto [ok, scratch] = reader->readRecord(); + ASSERT_TRUE(ok); + ASSERT_EQ(scratch, payload); + } +} } // namespace DB::PS::V3::tests diff --git a/dbms/src/Storages/SelectQueryInfo.cpp b/dbms/src/Storages/SelectQueryInfo.cpp index 9b5dbf0d30f..4569b25ef35 100644 --- a/dbms/src/Storages/SelectQueryInfo.cpp +++ b/dbms/src/Storages/SelectQueryInfo.cpp @@ -30,6 +30,7 @@ SelectQueryInfo::SelectQueryInfo(const SelectQueryInfo & rhs) , dag_query(rhs.dag_query != nullptr ? std::make_unique(*rhs.dag_query) : nullptr) , req_id(rhs.req_id) , keep_order(rhs.keep_order) + , is_fast_scan(rhs.is_fast_scan) {} SelectQueryInfo::SelectQueryInfo(SelectQueryInfo && rhs) noexcept @@ -39,6 +40,7 @@ SelectQueryInfo::SelectQueryInfo(SelectQueryInfo && rhs) noexcept , dag_query(std::move(rhs.dag_query)) , req_id(std::move(rhs.req_id)) , keep_order(rhs.keep_order) + , is_fast_scan(rhs.is_fast_scan) {} } // namespace DB diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 0807647aa26..4734076dff0 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -52,6 +52,7 @@ struct SelectQueryInfo std::string req_id; bool keep_order = true; + bool is_fast_scan = false; SelectQueryInfo(); ~SelectQueryInfo(); diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 485c0057135..6eb6a16736b 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -759,7 +759,7 @@ BlockInputStreams StorageDeltaMerge::read( rs_operator, req_id, query_info.keep_order, - /* is_fast_mode */ tidb_table_info.tiflash_mode == TiDB::TiFlashMode::Fast, // read in normal mode or read in fast mode + /* is_fast_scan */ query_info.is_fast_scan, max_block_size, parseSegmentSet(select_query.segment_expression_list), extra_table_id_index); diff --git a/dbms/src/Storages/Transaction/CollatorUtils.h b/dbms/src/Storages/Transaction/CollatorUtils.h index c757ade8043..3f53791276c 100644 --- a/dbms/src/Storages/Transaction/CollatorUtils.h +++ b/dbms/src/Storages/Transaction/CollatorUtils.h @@ -16,7 +16,9 @@ #include #include +#include +#include #include namespace DB @@ -31,9 +33,9 @@ ALWAYS_INLINE inline int signum(T val) // Check equality is much faster than other comparison. // - check size first // - return 0 if equal else 1 -FLATTEN_INLINE_PURE inline int RawStrEqualCompare(const std::string_view & lhs, const std::string_view & rhs) +FLATTEN_INLINE_PURE static inline int RawStrEqualCompare(const std::string_view & lhs, const std::string_view & rhs) { - return StringRef(lhs) == StringRef(rhs) ? 0 : 1; + return mem_utils::IsStrViewEqual(lhs, rhs) ? 0 : 1; } // Compare str view by memcmp @@ -91,4 +93,55 @@ FLATTEN_INLINE_PURE inline StringRef BinCollatorSortKey(const char * s, size_t l return StringRef(s, length); } } + +// Loop columns and invoke callback for each pair. +// Remove last zero byte. +template +FLATTEN_INLINE static inline void LoopTwoColumns( + const Chars & a_data, + const Offsets & a_offsets, + const Chars & b_data, + const Offsets & b_offsets, + size_t size, + F && func) +{ + uint64_t a_prev_offset = 0; + uint64_t b_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + auto a_size = a_offsets[i] - a_prev_offset; + auto b_size = b_offsets[i] - b_prev_offset; + + // Remove last zero byte. + func({reinterpret_cast(&a_data[a_prev_offset]), a_size - 1}, + {reinterpret_cast(&b_data[b_prev_offset]), b_size - 1}, + i); + + a_prev_offset = a_offsets[i]; + b_prev_offset = b_offsets[i]; + } +} + +// Loop one column and invoke callback for each pair. +// Remove last zero byte. +template +FLATTEN_INLINE static inline void LoopOneColumn( + const Chars & a_data, + const Offsets & a_offsets, + size_t size, + F && func) +{ + uint64_t a_prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + auto a_size = a_offsets[i] - a_prev_offset; + + // Remove last zero byte. + func({reinterpret_cast(&a_data[a_prev_offset]), a_size - 1}, i); + a_prev_offset = a_offsets[i]; + } +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index cd1dc583a7c..ed107e99aba 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -62,19 +62,19 @@ Field GenDefaultField(const TiDB::ColumnInfo & col_info) case TiDB::CodecFlagCompactBytes: return Field(String()); case TiDB::CodecFlagFloat: - return Field(Float64(0)); + return Field(static_cast(0)); case TiDB::CodecFlagUInt: - return Field(UInt64(0)); + return Field(static_cast(0)); case TiDB::CodecFlagInt: - return Field(Int64(0)); + return Field(static_cast(0)); case TiDB::CodecFlagVarInt: - return Field(Int64(0)); + return Field(static_cast(0)); case TiDB::CodecFlagVarUInt: - return Field(UInt64(0)); + return Field(static_cast(0)); case TiDB::CodecFlagJson: return TiDB::genJsonNull(); case TiDB::CodecFlagDuration: - return Field(Int64(0)); + return Field(static_cast(0)); default: throw Exception("Not implemented codec flag: " + std::to_string(col_info.getCodecFlag()), ErrorCodes::LOGICAL_ERROR); } @@ -407,7 +407,7 @@ try size_t elems_size = elems_arr->size(); for (size_t i = 1; i <= elems_size; i++) { - elems.push_back(std::make_pair(elems_arr->getElement(i - 1), Int16(i))); + elems.push_back(std::make_pair(elems_arr->getElement(i - 1), static_cast(i))); } } /// need to do this check for forward compatibility @@ -773,37 +773,6 @@ catch (const Poco::Exception & e) DB::Exception(e)); } -String TiFlashModeToString(TiFlashMode tiflash_mode) -{ - switch (tiflash_mode) - { - case TiFlashMode::Normal: - return ""; - case TiFlashMode::Fast: - return "fast"; - default: - LOG_FMT_WARNING(&Poco::Logger::get("TiDB"), "TiFlashModeToString with invalid tiflash mode {}", tiflash_mode); - return ""; - } -} - -TiFlashMode parseTiFlashMode(String mode_str) -{ - if (mode_str.empty()) - { - return TiFlashMode::Normal; - } - else if (mode_str == "fast") - { - return TiFlashMode::Fast; - } - else - { - throw DB::Exception( - std::string(__PRETTY_FUNCTION__) - + " ParseTiFlashMode Failed. mode " + mode_str + " is unvalid, please set mode as fast/normal"); - } -} /////////////////////// ////// TableInfo ////// /////////////////////// @@ -872,8 +841,6 @@ try json->set("tiflash_replica", replica_info.getJSONObject()); - json->set("tiflash_mode", std::string(TiFlashModeToString(tiflash_mode))); - json->stringify(buf); return buf.str(); @@ -960,14 +927,6 @@ try replica_info.deserialize(replica_obj); } } - if (obj->has("tiflash_mode")) - { - auto mode = obj->getValue("tiflash_mode"); - if (!mode.empty()) - { - tiflash_mode = parseTiFlashMode(mode); - } - } if (is_common_handle && index_infos.size() != 1) { throw DB::Exception( @@ -1138,7 +1097,7 @@ TableInfoPtr TableInfo::producePartitionTableInfo(TableID table_or_partition_id, String genJsonNull() { // null - const static String null({char(DB::TYPE_CODE_LITERAL), char(DB::LITERAL_NIL)}); + const static String null({static_cast(DB::TYPE_CODE_LITERAL), static_cast(DB::LITERAL_NIL)}); return null; } diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index 0f136786fd5..9bd78abeed3 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -197,10 +197,19 @@ struct ColumnInfo #ifdef M #error "Please undefine macro M first." #endif -#define M(f, v) \ - inline bool has##f##Flag() const { return (flag & (v)) != 0; } \ - inline void set##f##Flag() { flag |= (v); } \ - inline void clear##f##Flag() { flag &= (~(v)); } +#define M(f, v) \ + inline bool has##f##Flag() const \ + { \ + return (flag & (v)) != 0; \ + } \ + inline void set##f##Flag() \ + { \ + flag |= (v); \ + } \ + inline void clear##f##Flag() \ + { \ + flag &= (~(v)); \ + } COLUMN_FLAGS(M) #undef M @@ -334,12 +343,6 @@ struct IndexInfo bool is_global; }; -enum class TiFlashMode -{ - Normal, - Fast, -}; - struct TableInfo { TableInfo() = default; @@ -389,8 +392,6 @@ struct TableInfo // The TiFlash replica info persisted by TiDB TiFlashReplicaInfo replica_info; - TiFlashMode tiflash_mode = TiFlashMode::Normal; - ::TiDB::StorageEngine engine_type = ::TiDB::StorageEngine::UNSPECIFIED; ColumnID getColumnID(const String & name) const; @@ -423,7 +424,4 @@ tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci); ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); ColumnInfo toTiDBColumnInfo(const tipb::ColumnInfo & tipb_column_info); -String TiFlashModeToString(TiFlashMode tiflash_mode); -TiFlashMode parseTiFlashMode(String mode_str); - } // namespace TiDB diff --git a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp index 871153cb0e9..76b084f6de1 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp @@ -143,44 +143,44 @@ try {StmtCase{ 1145, // R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","partition":null,"tiflash_mode":"fast"})json", // - R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":1145,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"fast","tiflash_replica":{"Count":0},"update_timestamp":0}'))stmt", // + R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","partition":null})json", // + R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":1145,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":0}'))stmt", // }, StmtCase{ 2049, // R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null,"tiflash_mode":""})json", // - R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // + R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null})json", // + R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // }, StmtCase{ 31, // R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // - R"stmt(CREATE TABLE `db1`.`simple_t`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // + R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db1`.`simple_t`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // }, StmtCase{ 33, // R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // - R"stmt(CREATE TABLE `db2`.`pk_t`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}'))stmt", // + R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db2`.`pk_t`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}'))stmt", // }, StmtCase{ 35, // R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // - R"stmt(CREATE TABLE `db1`.`not_null_t`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}'))stmt", // + R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db1`.`not_null_t`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}'))stmt", // }, StmtCase{ 37, // R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", - R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // - R"stmt(CREATE TABLE `db2`.`mytable`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}'))stmt", // + R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db2`.`mytable`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}'))stmt", // }, StmtCase{ 32, // R"json({"id":1,"db_name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1,"tiflash_mode":""})json", // - R"stmt(CREATE TABLE `test`.`range_part_t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}'))stmt", // + R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // + R"stmt(CREATE TABLE `test`.`range_part_t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}'))stmt", // }}; for (const auto & c : cases) diff --git a/dbms/src/Storages/Transaction/tests/gtest_tidb_collator.cpp b/dbms/src/Storages/Transaction/tests/gtest_tidb_collator.cpp index 0ba71d6bbbb..67e23ee7cbc 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_tidb_collator.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_tidb_collator.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -225,7 +226,7 @@ void testCollator() ColumnString::Chars_t strs; ColumnString::Offsets offsets; - std::vector res; + PaddedPODArray res; { // init data ColumnString::Offset current_new_offset = 0; for (const auto & inner_c : inner_cases) diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index af7169e7c47..85287bebc0f 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -543,11 +543,6 @@ void SchemaBuilder::applyDiff(const SchemaDiff & diff) applySetTiFlashReplica(db_info, diff.table_id); break; } - case SchemaActionType::SetTiFlashMode: - { - applySetTiFlashMode(db_info, diff.table_id); - break; - } default: { if (diff.type < SchemaActionType::MaxRecognizedType) @@ -1265,76 +1260,6 @@ void SchemaBuilder::applySetTiFlashReplicaOnPhysicalTable( LOG_FMT_INFO(log, "Updated replica info for {}", name_mapper.debugCanonicalName(*db_info, table_info)); } - -template -void SchemaBuilder::applySetTiFlashMode(const TiDB::DBInfoPtr & db_info, TableID table_id) -{ - auto latest_table_info = getter.getTableInfo(db_info->id, table_id); - - if (unlikely(latest_table_info == nullptr)) - { - throw TiFlashException(fmt::format("miss table in TiKV : {}", table_id), Errors::DDL::StaleSchema); - } - - auto & tmt_context = context.getTMTContext(); - auto storage = tmt_context.getStorages().get(latest_table_info->id); - if (unlikely(storage == nullptr)) - { - throw TiFlashException(fmt::format("miss table in TiFlash : {}", name_mapper.debugCanonicalName(*db_info, *latest_table_info)), - Errors::DDL::MissingTable); - } - - applySetTiFlashModeOnLogicalTable(db_info, latest_table_info, storage); -} - -template -void SchemaBuilder::applySetTiFlashModeOnLogicalTable( - const TiDB::DBInfoPtr & db_info, - const TiDB::TableInfoPtr & table_info, - const ManageableStoragePtr & storage) -{ - applySetTiFlashModeOnPhysicalTable(db_info, table_info, storage); - - if (table_info->isLogicalPartitionTable()) - { - auto & tmt_context = context.getTMTContext(); - for (const auto & part_def : table_info->partition.definitions) - { - auto new_part_table_info = table_info->producePartitionTableInfo(part_def.id, name_mapper); - auto part_storage = tmt_context.getStorages().get(new_part_table_info->id); - if (unlikely(part_storage == nullptr)) - { - throw TiFlashException(fmt::format("miss table in TiFlash : {}", name_mapper.debugCanonicalName(*db_info, *new_part_table_info)), - Errors::DDL::MissingTable); - } - applySetTiFlashModeOnPhysicalTable(db_info, new_part_table_info, part_storage); - } - } -} - - -template -void SchemaBuilder::applySetTiFlashModeOnPhysicalTable( - const TiDB::DBInfoPtr & db_info, - const TiDB::TableInfoPtr & latest_table_info, - const ManageableStoragePtr & storage) -{ - if (storage->getTableInfo().tiflash_mode == latest_table_info->tiflash_mode) - return; - - TiDB::TableInfo table_info = storage->getTableInfo(); - table_info.tiflash_mode = latest_table_info->tiflash_mode; - table_info.schema_version = target_version; - AlterCommands commands; - - LOG_FMT_INFO(log, "Updating tiflash mode for {} to {}", name_mapper.debugCanonicalName(*db_info, table_info), TiFlashModeToString(table_info.tiflash_mode)); - - auto alter_lock = storage->lockForAlter(getThreadName()); - storage->alterFromTiDB(alter_lock, commands, name_mapper.mapDatabaseName(*db_info), table_info, name_mapper, context); - LOG_FMT_INFO(log, "Updated tiflash mode for {} to {}", name_mapper.debugCanonicalName(*db_info, table_info), TiFlashModeToString(table_info.tiflash_mode)); -} - - template void SchemaBuilder::syncAllSchema() { @@ -1404,8 +1329,6 @@ void SchemaBuilder::syncAllSchema() applyRenameLogicalTable(db, table, storage); /// Update replica info if needed. applySetTiFlashReplicaOnLogicalTable(db, table, storage); - /// Update tiflash mode if needed. - applySetTiFlashModeOnLogicalTable(db, table, storage); /// Alter if needed. applyAlterLogicalTable(db, table, storage); LOG_FMT_DEBUG(log, "Table {} synced during sync all schemas", name_mapper.debugCanonicalName(*db, *table)); diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index 827203a682f..461d7ff9c12 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -89,10 +89,6 @@ struct SchemaBuilder void applySetTiFlashReplica(const TiDB::DBInfoPtr & db_info, TableID table_id); void applySetTiFlashReplicaOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); void applySetTiFlashReplicaOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - - void applySetTiFlashMode(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applySetTiFlashModeOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applySetTiFlashModeOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); }; } // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index 72fd00678f7..b03ea01e5bb 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -95,13 +95,11 @@ enum class SchemaActionType : Int8 AlterNoCacheTable = 59, CreateTables = 60, ActionMultiSchemaChange = 61, - SetTiFlashMode = 62, - // If we supporte new type from TiDB. // MaxRecognizedType also needs to be changed. // It should always be equal to the maximum supported type + 1 - MaxRecognizedType = 63, + MaxRecognizedType = 62, }; struct AffectedOption diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 536cc97081f..9cb8ec8c146 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -52,6 +52,7 @@ add_library (common ${SPLIT_SHARED} src/crc64.cpp src/simd.cpp src/detect_features.cpp + src/avx2_mem_utils_impl.cpp include/common/types.h include/common/DateLUT.h @@ -159,6 +160,7 @@ endif () if (TIFLASH_ENABLE_AVX_SUPPORT) # https://gcc.gnu.org/onlinedocs/gcc/x86-Options.html set_source_files_properties(src/mem_utils_avx2.cpp APPEND COMPILE_FLAGS "-mavx -mavx2") + set_source_files_properties(src/avx2_mem_utils_impl.cpp APPEND COMPILE_FLAGS "-mavx -mavx2") if (TIFLASH_COMPILER_VPCLMULQDQ_SUPPORT) set_source_files_properties(src/crc64_avx2.cpp APPEND COMPILE_FLAGS "-mavx2 -mpclmul -mvpclmulqdq -Wno-ignored-attributes") diff --git a/libs/libcommon/include/common/StringRef.h b/libs/libcommon/include/common/StringRef.h index 8853825cbc4..f43c18370d3 100644 --- a/libs/libcommon/include/common/StringRef.h +++ b/libs/libcommon/include/common/StringRef.h @@ -82,6 +82,9 @@ constexpr const inline char empty_string_ref_addr{}; using StringRefs = std::vector; +// According to https://github.com/pingcap/tiflash/pull/5658 +// - if size of memory area is bigger than 1M, instructions about avx512 may begin to get better results +// - otherwise, use `std::string_view == std::string_view` or `mem_utils::avx2_mem_equal`(under x86-64 with avx2) inline bool operator==(StringRef lhs, StringRef rhs) { if (lhs.size != rhs.size) diff --git a/libs/libcommon/include/common/avx2_mem_utils.h b/libs/libcommon/include/common/avx2_mem_utils.h new file mode 100644 index 00000000000..822fb4b7a40 --- /dev/null +++ b/libs/libcommon/include/common/avx2_mem_utils.h @@ -0,0 +1,474 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 + +namespace mem_utils::details +{ + +// remove tail `1`: `111000` -> `110000` +template +ALWAYS_INLINE static inline T clear_rightmost_bit_one(const T value) +{ + assert(value != 0); + + return value & (value - 1); +} + +// get count of tail `0` bit, same as get index of rightmost bit `1` +// - `11000` -> `3` means the right most `1` is at index `3` +ALWAYS_INLINE static inline uint32_t rightmost_bit_one_index(const uint32_t value) +{ + assert(value != 0); + return _tzcnt_u32(value); +} + +using Block32 = __m256i; +constexpr int BLOCK32_SIZE = sizeof(Block32); +using Block16 = __m128i; +constexpr int BLOCK16_SIZE = sizeof(Block16); +constexpr auto Block32Mask = std::numeric_limits::max(); +constexpr uint32_t Block16Mask = std::numeric_limits::max(); +constexpr auto AVX2_UNROLL_NUM = 4; + +// `N` is `1U << xxx` +#define OFFSET_FROM_ALIGNED(ADDR, N) ((ADDR) % (N)) // (ADDR) & (N-1) +#define ALIGNED_ADDR(ADDR, N) ((ADDR) / (N) * (N)) // (ADDR) & (-N) + +template +FLATTEN_INLINE_PURE static inline T read(const void * data) +{ + T val = *reinterpret_cast(data); + return val; +} +FLATTEN_INLINE_PURE static inline Block32 load_block32(const void * p) +{ + return _mm256_loadu_si256(reinterpret_cast(p)); +} +FLATTEN_INLINE_PURE static inline Block16 load_block16(const void * p) +{ + return _mm_loadu_si128(reinterpret_cast(p)); +} +FLATTEN_INLINE_PURE static inline uint32_t get_block32_cmp_eq_mask(const void * p1, const void * p2) +{ + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpeq_epi8(load_block32(p1), load_block32(p2))); + return mask; +} +FLATTEN_INLINE_PURE static inline uint32_t get_block16_cmp_eq_mask(const void * p1, const void * p2) +{ + uint32_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(load_block16(p1), load_block16(p2))); + return mask; +} +FLATTEN_INLINE_PURE static inline bool check_block32_eq(const char * a, const char * b) +{ + auto data = _mm256_xor_si256( + load_block32(a), + load_block32(b)); + return 0 != _mm256_testz_si256(data, data); +} +FLATTEN_INLINE_PURE static inline int cmp_block1(const void * p1, const void * p2) +{ + return int32_t(read(p1)) - int32_t(read(p2)); +} + +FLATTEN_INLINE_PURE static inline int cmp_block8(const void * p1, const void * p2) +{ + // the left most bit may be 1, use std::memcmp(,,8) to use `sbb` + /* + bswap rcx + bswap rdx + xor eax, eax + cmp rcx, rdx + seta al + sbb eax, 0 + */ + return std::memcmp(p1, p2, 8); +} + +FLATTEN_INLINE_PURE static inline int cmp_block16(const char * p1, const char * p2) +{ + uint32_t mask = get_block16_cmp_eq_mask(p1, p2); // mask is up to 0xffff + mask -= Block16Mask; + if (unlikely(mask != 0)) + { + auto pos = rightmost_bit_one_index(mask); + return cmp_block1(p1 + pos, p2 + pos); + } + return 0; +} +FLATTEN_INLINE_PURE static inline int cmp_block32(const char * p1, const char * p2) +{ + uint32_t mask = get_block32_cmp_eq_mask(p1, p2); // mask is up to 0xffffffff + mask -= Block32Mask; + if (unlikely(mask != 0)) + { + auto pos = rightmost_bit_one_index(mask); + return cmp_block1(p1 + pos, p2 + pos); + } + return 0; +} + +template +FLATTEN_INLINE_PURE static inline bool check_block32x4_eq(const char * a, const char * b) +{ + if constexpr (use_vptest_instr) + { + auto all_ones = _mm256_set1_epi8(0xFF); + Block32 data = all_ones; + for (size_t i = 0; i < AVX2_UNROLL_NUM; ++i) + data = _mm256_and_si256(data, _mm256_cmpeq_epi8(load_block32(a + i * BLOCK32_SIZE), load_block32(b + i * BLOCK32_SIZE))); + return 0 != _mm256_testc_si256(data, all_ones); + } + else + { + uint32_t mask = Block32Mask; + for (size_t i = 0; i < AVX2_UNROLL_NUM; ++i) + mask &= get_block32_cmp_eq_mask(a + i * BLOCK32_SIZE, b + i * BLOCK32_SIZE); + return mask == Block32Mask; + } +} + +FLATTEN_INLINE_PURE static inline int cmp_block32x4(const char * a, const char * b) +{ + if (check_block32x4_eq(a, b)) + return 0; + for (size_t i = 0; i < AVX2_UNROLL_NUM - 1; ++i) + { + if (auto ret = cmp_block32(a + i * BLOCK32_SIZE, (b + i * BLOCK32_SIZE)); ret) + return ret; + } + return cmp_block32(a + (AVX2_UNROLL_NUM - 1) * BLOCK32_SIZE, (b + (AVX2_UNROLL_NUM - 1) * BLOCK32_SIZE)); +} +FLATTEN_INLINE_PURE static inline uint32_t swap_u32(uint32_t val) +{ + return __builtin_bswap32(val); +} +FLATTEN_INLINE_PURE static inline uint64_t swap_u64(uint64_t val) +{ + return __builtin_bswap64(val); +} + +[[maybe_unused]] FLATTEN_INLINE_PURE static inline uint32_t read_u32_swap(const void * data) +{ + return swap_u32(read(data)); +} + +[[maybe_unused]] FLATTEN_INLINE_PURE static inline uint64_t read_u64_swap(const void * data) +{ + return swap_u64(read(data)); +} + +// ref: https://github.com/lattera/glibc/blob/master/sysdeps/x86_64/multiarch/memcmp-avx2-movbe.S +FLATTEN_INLINE_PURE static inline int avx2_mem_cmp(const char * p1, const char * p2, size_t n) +{ + constexpr size_t loop_block32x4_size = AVX2_UNROLL_NUM * BLOCK32_SIZE; + + // n <= 32 + if (likely(n <= BLOCK32_SIZE)) + { + if (unlikely(n < 2)) + { + // 0~1 + if (n == 1) + return cmp_block1(p1, p2); + return 0; + } + else if (unlikely(n < 4)) + { + // 2~3 + using T = uint16_t; + + // load 2 bytes to one int32: [0, 0, p1, p0] + // shift left one byte: [0, p1, p0, 0] + // reverse swap: [0, p0, p1, 0] + // add one byte from high addr: [0, p0, p1, p2] + // the left most bit is always 0, it's safe to use subtraction directly + int32_t a = read(p1); + int32_t b = read(p2); + a <<= sizeof(uint8_t) * 8; + b <<= sizeof(uint8_t) * 8; + a = swap_u32(a); + b = swap_u32(b); + a |= read(p1 + n - sizeof(uint8_t)); + b |= read(p2 + n - sizeof(uint8_t)); + + return a - b; + } + else if (unlikely(n <= 8)) + { + // 4~8 + using T = uint32_t; + + // load high 4 bytes to one uint64: [0, 0, 0, 0, pn, p_n-1, p_n-2, p_n-3] + // shift left 4 byte: [pn, p_n-1, p_n-2, p_n-3, 0, 0, 0, 0] + // add low 4 bytes from high addr: [pn, p_n-1, p_n-2, p_n-3, p3, p2, p1, p0] + uint64_t a = read(p1 + n - sizeof(T)); + uint64_t b = read(p2 + n - sizeof(T)); + a <<= sizeof(T) * 8; + b <<= sizeof(T) * 8; + a |= read(p1); + b |= read(p2); + + return cmp_block8(&a, &b); + } + else if (likely(n <= 16)) + { + // 9~16 + if (auto ret = cmp_block8(p1, p2); ret) + return ret; + return cmp_block8(p1 + n - 8, p2 + n - 8); + } + else + { + // 17~32 + if (auto ret = cmp_block16(p1, p2); ret) + return ret; + return cmp_block16(p1 + n - BLOCK16_SIZE, p2 + n - BLOCK16_SIZE); + } + } + // 8 * 32 < n + if (unlikely(8 * BLOCK32_SIZE < n)) + { + // check first block + + if (auto ret = cmp_block32(p1, p2); unlikely(ret)) + return ret; + { + // align addr of one data pointer + auto offset = BLOCK32_SIZE - OFFSET_FROM_ALIGNED(size_t(p2), BLOCK32_SIZE); + p1 += offset; + p2 += offset; + n -= offset; + } + + for (; n >= loop_block32x4_size;) + { + if (auto ret = cmp_block32x4(p1, p2); unlikely(ret)) + return ret; + + n -= loop_block32x4_size; + p1 += loop_block32x4_size; + p2 += loop_block32x4_size; + } + // n < 4 * 32 + if (unlikely(n <= BLOCK32_SIZE)) + { + // n <= 32 + return cmp_block32(p1 + n - BLOCK32_SIZE, p2 + n - BLOCK32_SIZE); + } + // 32 < n < 4 * 32 + } + + assert(BLOCK32_SIZE < n); + + if (unlikely(n <= 2 * BLOCK32_SIZE)) + { + // 32 < n <= 2 * 32 + if (auto ret = cmp_block32(p1, p2); unlikely(ret)) + return ret; + return cmp_block32(p1 + n - BLOCK32_SIZE, p2 + n - BLOCK32_SIZE); + } + if (unlikely(n <= 4 * BLOCK32_SIZE)) + { + // 2 * 32 < n <= 4 * 32 + if (auto ret = cmp_block32(p1, p2); unlikely(ret)) + return ret; + if (auto ret = cmp_block32(p1 + BLOCK32_SIZE, p2 + BLOCK32_SIZE); unlikely(ret)) + return ret; + p1 = p1 + n - 2 * BLOCK32_SIZE; + p2 = p2 + n - 2 * BLOCK32_SIZE; + if (auto ret = cmp_block32(p1, p2); unlikely(ret)) + return ret; + return cmp_block32(p1 + BLOCK32_SIZE, p2 + BLOCK32_SIZE); + } + + assert(n <= 8 * BLOCK32_SIZE); + + { + // 4 * 32 < n <= 8 * 32 + if (auto ret = cmp_block32x4(p1, p2); unlikely(ret)) + return ret; + return cmp_block32x4(p1 + n - loop_block32x4_size, p2 + n - loop_block32x4_size); + } +} + +FLATTEN_INLINE_PURE static inline bool avx2_mem_equal(const char * p1, const char * p2, size_t n) +{ + constexpr size_t loop_block32x4_size = AVX2_UNROLL_NUM * BLOCK32_SIZE; + + // n <= 32 + if (likely(n <= BLOCK32_SIZE)) + { +#ifdef M + static_assert(false, "`M` is defined"); +#else +#define M(x) \ + case (x): \ + { \ + return mem_utils::memcmp_eq_fixed_size<(x)>(p1, p2); \ + } +#endif + switch (n) + { + M(0); + M(1); + M(2); + M(3); + M(4); + M(5); + M(6); + M(7); + M(8); + M(9); + M(10); + M(11); + M(12); + M(13); + M(14); + M(15); + M(16); + default: + { + // 17~32 + if (!mem_utils::memcmp_eq_fixed_size(p1, p2)) + return false; + if (!mem_utils::memcmp_eq_fixed_size(p1 + n - BLOCK16_SIZE, p2 + n - BLOCK16_SIZE)) + return false; + return true; + } + } +#undef M + +// an optional way to check small str +#if defined(AVX2_MEM_EQ_NORMAL_IF_ELSE) + if (unlikely(n < 2)) + { + // 0~1 + if (n == 1) + return p1[0] == p2[0]; + return true; + } + else if (unlikely(n <= 4)) + { + // 2~4 + auto a1 = *reinterpret_cast(p1); + auto b1 = *reinterpret_cast(p2); + auto a2 = *reinterpret_cast(p1 + n - 2); + auto b2 = *reinterpret_cast(p2 + n - 2); + return (a1 == b1) & (a2 == b2); + } + else if (unlikely(n <= 8)) + { + // 5~8 + auto a1 = *reinterpret_cast(p1); + auto b1 = *reinterpret_cast(p2); + auto a2 = *reinterpret_cast(p1 + n - 4); + auto b2 = *reinterpret_cast(p2 + n - 4); + return (a1 == b1) & (a2 == b2); + } + else if (unlikely(n <= 16)) + { + // 9~16 + auto a1 = *reinterpret_cast(p1); + auto b1 = *reinterpret_cast(p2); + auto a2 = *reinterpret_cast(p1 + n - 8); + auto b2 = *reinterpret_cast(p2 + n - 8); + return (a1 == b1) & (a2 == b2); + } + else + { + // 17~32 + if (!memcmp_eq_fixed_size(p1, p2)) + return false; + if (!memcmp_eq_fixed_size(p1 + n - BLOCK16_SIZE, p2 + n - BLOCK16_SIZE)) + return false; + return true; + } +#endif + } + + // 8 * 32 < n + if (unlikely(8 * BLOCK32_SIZE < n)) + { + // check first block + if (unlikely(!check_block32_eq(p1, p2))) + return false; + { + // align addr of one data pointer + auto offset = BLOCK32_SIZE - OFFSET_FROM_ALIGNED(size_t(p2), BLOCK32_SIZE); + p1 += offset; + p2 += offset; + n -= offset; + } + + for (; n >= loop_block32x4_size;) + { + if (unlikely(!check_block32x4_eq(p1, p2))) + return false; + + n -= loop_block32x4_size; + p1 += loop_block32x4_size; + p2 += loop_block32x4_size; + } + // n < 4 * 32 + if (unlikely(n <= BLOCK32_SIZE)) + { + // n <= 32 + return check_block32_eq(p1 + n - BLOCK32_SIZE, p2 + n - BLOCK32_SIZE); + } + // 32 < n < 4 * 32 + } + + assert(BLOCK32_SIZE < n); + + if (unlikely(n <= 2 * BLOCK32_SIZE)) + { + // 32 < n <= 2 * 32 + if (unlikely(!check_block32_eq(p1, p2))) + return false; + return check_block32_eq(p1 + n - BLOCK32_SIZE, p2 + n - BLOCK32_SIZE); + } + if (unlikely(n <= 4 * BLOCK32_SIZE)) + { + // 2 * 32 < n <= 4 * 32 + if (unlikely(!check_block32_eq(p1, p2))) + return false; + if (unlikely(!check_block32_eq(p1 + BLOCK32_SIZE, p2 + BLOCK32_SIZE))) + return false; + p1 = p1 + n - 2 * BLOCK32_SIZE; + p2 = p2 + n - 2 * BLOCK32_SIZE; + if (unlikely(!check_block32_eq(p1, p2))) + return false; + return check_block32_eq(p1 + BLOCK32_SIZE, p2 + BLOCK32_SIZE); + } + + assert(n <= 8 * BLOCK32_SIZE); + + { + // 4 * 32 < n <= 8 * 32 + if (unlikely(!check_block32x4_eq(p1, p2))) + return false; + return check_block32x4_eq(p1 + n - loop_block32x4_size, p2 + n - loop_block32x4_size); + } +} +} // namespace mem_utils::details diff --git a/libs/libcommon/include/common/avx2_strstr.h b/libs/libcommon/include/common/avx2_strstr.h new file mode 100644 index 00000000000..f8d44702f93 --- /dev/null +++ b/libs/libcommon/include/common/avx2_strstr.h @@ -0,0 +1,282 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 mem_utils::details +{ + +FLATTEN_INLINE_PURE static inline uint32_t get_block32_cmp_eq_mask( + const void * s, + const Block32 & check_block) +{ + /* + vpcmpeqb ymm0, ymm0, ymmword ptr [...] + */ + // `_mm256_loadu_si256` and `_mm256_load_si256` are same in such case + const auto block = load_block32(s); + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpeq_epi8(block, check_block)); + return mask; +} + +template +ALWAYS_INLINE static inline bool check_aligned_block32_may_exceed(const char * src, ssize_t n, const char *& res, const Block32 & check_block, F && callback) +{ + auto mask = get_block32_cmp_eq_mask(src, check_block); + for (; mask;) + { + auto c = rightmost_bit_one_index(mask); + // check boundary + if (c >= n) + { + res = nullptr; + return true; + } + // + const auto * t = c + src; + if (callback(t)) + { + res = t; + return true; + } + mask = clear_rightmost_bit_one(mask); + } + return false; +} + +template +ALWAYS_INLINE static inline bool check_block32x1(const char * src, const char *& res, const Block32 & check_block, F && callback) +{ + auto mask = get_block32_cmp_eq_mask(src, check_block); + for (; mask;) + { + const auto * t = src + rightmost_bit_one_index(mask); + if (callback(t)) + { + res = t; + return true; + } + mask = clear_rightmost_bit_one(mask); + } + return false; +} + +template +ALWAYS_INLINE static inline bool check_block32x4(const char * src, const char *& res, const Block32 & check_block, F && callback) +{ + { + uint32_t data{}; + for (size_t i = 0; i < AVX2_UNROLL_NUM; ++i) + data |= get_block32_cmp_eq_mask( + src + BLOCK32_SIZE * i, + check_block); + + if (data) + { + // there must be matched mask + } + else + { + return false; + } + } + + for (size_t i = 0; i < AVX2_UNROLL_NUM; ++i) + { + const auto * start = src + BLOCK32_SIZE * i; + auto mask = get_block32_cmp_eq_mask( + start, + check_block); + for (; mask;) + { + auto c = rightmost_bit_one_index(mask); + const auto * t = c + start; + if (callback(t)) + { + res = t; + return true; + } + mask = clear_rightmost_bit_one(mask); + } + } + return false; +}; + +template +ALWAYS_INLINE static inline const char * avx2_strstr_impl(const char * src, const char target, ssize_t n, F && callback) +{ + assert(n >= 1); + + const char * res = nullptr; + const auto check_block32 = _mm256_set1_epi8(target); + + // align address to 32 for better performance + // memory allocator will always alloc memory aligned to `Page Size`(usually 4K, one Block `512B` at least) from system + // if there is valid data at address S, then it is safe to visit address [ALIGN_TO_PAGE_SIZE(S), ALIGN_TO_PAGE_SIZE(S)+PAGE_SIZE). + if (uint8_t offset = OFFSET_FROM_ALIGNED(size_t(src), BLOCK32_SIZE); offset != 0) + { + // align to 32 + src = reinterpret_cast(ALIGNED_ADDR(size_t(src), BLOCK32_SIZE)); + + // load block 32 from new aligned address may cause false positives when using `AddressSanitizer` because asan will provide a malloc()/free() alternative and detect memory visitation. + // generally it's safe to visit address which won't cross page boundary. + + // right shift offset to remove useless mask bit + auto mask = get_block32_cmp_eq_mask(src, check_block32) >> offset; + + for (; mask;) + { + auto c = rightmost_bit_one_index(mask); + if (c >= n) + return nullptr; + const auto * t = c + src + offset; // add offset + if (callback(t)) + return t; + mask = clear_rightmost_bit_one(mask); + } + + n -= BLOCK32_SIZE - offset; + + if (n <= 0) + { + return nullptr; + } + + src += BLOCK32_SIZE; + } + + assert(size_t(src) % BLOCK32_SIZE == 0); + + for (; (n >= AVX2_UNROLL_NUM * BLOCK32_SIZE);) + { + if (check_block32x4(src, res, check_block32, callback)) + return res; + src += AVX2_UNROLL_NUM * BLOCK32_SIZE, n -= AVX2_UNROLL_NUM * BLOCK32_SIZE; + } + + assert(n < AVX2_UNROLL_NUM * BLOCK32_SIZE); + + for (; (n >= BLOCK32_SIZE);) + { + if (check_block32x1(src, res, check_block32, callback)) + return res; + n -= BLOCK32_SIZE, src += BLOCK32_SIZE; + } + + if (unlikely(n == 0)) + return nullptr; + + check_aligned_block32_may_exceed(src, n, res, check_block32, callback); + return res; +} + +inline const char * avx2_strstr_impl_genetic(const char * src, size_t n, const char * needle, size_t k) +{ + return avx2_strstr_impl(src, needle[0], n - k + 1, [&](const char * s) -> bool { + return avx2_mem_equal(s, needle, k); + }); +} + +ALWAYS_INLINE static inline const char * avx2_strstr_impl(const char * src, size_t n, const char * needle, size_t k) +{ +#ifdef M + static_assert(false, "`M` is defined"); +#else +#define M(x) \ + case (x): \ + { \ + return avx2_strstr_impl(src, needle[0], n - (x) + 1, [&](const char * s) -> bool { \ + return mem_utils::memcmp_eq_fixed_size(s, needle); \ + }); \ + } +#endif + + if (unlikely(n < k)) + { + return nullptr; + } + + switch (k) + { + case 0: + { + return src; + } + case 1: + { + return avx2_strstr_impl( + src, + needle[0], + n, + [&](const char *) constexpr { + return true; + }); + } + M(2); + M(3); + M(4); + M(5); + M(6); + M(7); + M(8); + M(9); + M(10); + M(11); + M(12); + M(13); + M(14); + M(15); + M(16); + default: + { + return avx2_strstr_impl_genetic(src, n, needle, k); + } + } +#undef M +} + +ALWAYS_INLINE static inline size_t avx2_strstr(const char * src, size_t n, const char * needle, size_t k) +{ +#if defined(ADDRESS_SANITIZER) + return std::string_view{src, n}.find({needle, k}); // memchr@plt -> bcmp@plt +#endif + + const auto * p = avx2_strstr_impl(src, n, needle, k); + return p ? p - src : std::string_view::npos; +} +ALWAYS_INLINE static inline size_t avx2_strstr(std::string_view src, std::string_view needle) +{ + return avx2_strstr(src.data(), src.size(), needle.data(), needle.size()); +} +ALWAYS_INLINE static inline const char * avx2_memchr(const char * src, size_t n, char target) +{ +#if defined(ADDRESS_SANITIZER) + return static_cast(std::memchr(src, target, n)); // memchr@plt +#endif + + if (unlikely(n < 1)) + { + return nullptr; + } + return avx2_strstr_impl( + src, + target, + n, + [&](const char *) constexpr { + return true; + }); +} +} // namespace mem_utils::details diff --git a/libs/libcommon/include/common/mem_utils.h b/libs/libcommon/include/common/mem_utils.h index baf3c8258a8..a440178d8a8 100644 --- a/libs/libcommon/include/common/mem_utils.h +++ b/libs/libcommon/include/common/mem_utils.h @@ -135,6 +135,10 @@ __attribute__((always_inline, pure)) inline bool memoryEqualSSE2(const char * p1 /// this function tries to utilize runtime available vectorization technology. /// it performs better than `std::memcmp`, especially for those OS with a /// relatively old libc. + +// According to https://github.com/pingcap/tiflash/pull/5658 +// - if size of memory area is bigger than 1M, instructions about avx512 may begin to get better results +// - otherwise, use `std::memcmp(p1,p2,n) == 0` or `mem_utils::avx2_mem_equal`(under x86-64 with avx2) __attribute__((always_inline, pure)) inline bool memoryEqual(const char * p1, const char * p2, size_t size) noexcept { using namespace common; diff --git a/libs/libcommon/include/common/mem_utils_opt.h b/libs/libcommon/include/common/mem_utils_opt.h new file mode 100644 index 00000000000..66057ccdd51 --- /dev/null +++ b/libs/libcommon/include/common/mem_utils_opt.h @@ -0,0 +1,119 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 + +#ifdef TIFLASH_ENABLE_AVX_SUPPORT + +#if defined(__AVX2__) + +#define TIFLASH_USE_AVX2_COMPILE_FLAG 1 + +// if cpp source file is compiled with flag `-mavx2`, it's recommended to use inline function for better performance. +#include +#include + +#endif + +#endif + +#ifdef TIFLASH_ENABLE_AVX_SUPPORT + +namespace mem_utils +{ + +// same function like `std::string_view::find` +// - return `-1` if failed to find `needle` in `src` +// - return `0` if size of `needle` is 0 +// - return the position where `needle` occur first +size_t avx2_strstr(std::string_view src, std::string_view needle); +size_t avx2_strstr(const char * src, size_t n, const char * needle, size_t k); + +// same function like `std::memchr` +// - return the first address pointer which equal to target `char` +// - reeurn `nullptr` if no match +const char * avx2_memchr(const char * src, size_t n, char target); + +// same function like `bcmp` or `std::memcmp(p1,p2,n) == 0` +bool avx2_mem_equal(const char * p1, const char * p2, size_t n); + +// same function like `std::memcmp` +int avx2_mem_cmp(const char * p1, const char * p2, size_t n); + +} // namespace mem_utils + +#endif + +namespace mem_utils +{ + +// same function like `std::string_view == std::string_view` +FLATTEN_INLINE_PURE static inline bool IsStrViewEqual(const std::string_view & lhs, const std::string_view & rhs) +{ + if (lhs.size() != rhs.size()) + return false; + +#ifdef TIFLASH_ENABLE_AVX_SUPPORT +#ifdef TIFLASH_USE_AVX2_COMPILE_FLAG + return mem_utils::details::avx2_mem_equal(lhs.data(), rhs.data(), lhs.size()); +#else + return mem_utils::avx2_mem_equal(lhs.data(), rhs.data(), lhs.size()); +#endif +#else + return 0 == std::memcmp(lhs.data(), rhs.data(), lhs.size()); +#endif +} + +// same function like `std::string_view.compare(std::string_view)` +FLATTEN_INLINE_PURE static inline int CompareStrView(const std::string_view & lhs, const std::string_view & rhs) +{ +#ifdef TIFLASH_ENABLE_AVX_SUPPORT + const size_t size = std::min(lhs.size(), rhs.size()); + +#ifdef TIFLASH_USE_AVX2_COMPILE_FLAG + int ret = mem_utils::details::avx2_mem_cmp(lhs.data(), rhs.data(), size); +#else + int ret = mem_utils::avx2_mem_cmp(lhs.data(), rhs.data(), size); +#endif + + if (ret == 0) + { + auto a = lhs.size(), b = rhs.size(); + ret = (a == b) ? 0 : (a < b ? -1 : 1); + } + return ret; +#else + return lhs.compare(rhs); +#endif +} + +// same function like `std::string_view::find(std::string_view)` +FLATTEN_INLINE_PURE static inline size_t StrFind(std::string_view src, std::string_view needle) +{ +#ifdef TIFLASH_ENABLE_AVX_SUPPORT +#ifdef TIFLASH_USE_AVX2_COMPILE_FLAG + return mem_utils::details::avx2_strstr(src, needle); +#else + return mem_utils::avx2_strstr(src, needle); +#endif +#else + return src.find(needle); +#endif +} + +} // namespace mem_utils \ No newline at end of file diff --git a/libs/libcommon/src/avx2_mem_utils_impl.cpp b/libs/libcommon/src/avx2_mem_utils_impl.cpp new file mode 100644 index 00000000000..9e66d645304 --- /dev/null +++ b/libs/libcommon/src/avx2_mem_utils_impl.cpp @@ -0,0 +1,51 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 + +#ifdef TIFLASH_ENABLE_AVX_SUPPORT + +#ifndef TIFLASH_USE_AVX2_COMPILE_FLAG +static_assert(false, "`libs/libcommon/src/avx2_mem_utils_impl.cpp` need flag `-mavx2` to inline related functions"); +#endif + +namespace mem_utils +{ +size_t avx2_strstr(const char * src, size_t n, const char * needle, size_t k) +{ + return details::avx2_strstr(src, n, needle, k); +} +size_t avx2_strstr(std::string_view src, std::string_view needle) +{ + return details::avx2_strstr(src, needle); +} + +bool avx2_mem_equal(const char * p1, const char * p2, size_t n) +{ + return details::avx2_mem_equal(p1, p2, n); +} + +int avx2_mem_cmp(const char * p1, const char * p2, size_t n) +{ + return details::avx2_mem_cmp(p1, p2, n); +} + +const char * avx2_memchr(const char * src, size_t n, char target) +{ + return details::avx2_memchr(src, n, target); +} + +} // namespace mem_utils + +#endif diff --git a/libs/libcommon/src/tests/CMakeLists.txt b/libs/libcommon/src/tests/CMakeLists.txt index 655d118c354..3383ee50da2 100644 --- a/libs/libcommon/src/tests/CMakeLists.txt +++ b/libs/libcommon/src/tests/CMakeLists.txt @@ -35,6 +35,7 @@ add_executable (gtests_libcommon gtest_json_test.cpp gtest_strong_typedef.cpp gtest_mem_utils.cpp + gtest_mem_utils_opt.cpp gtest_crc64.cpp gtest_logger.cpp gtest_arithmetic_overflow.cpp @@ -43,9 +44,11 @@ target_link_libraries (gtests_libcommon gtest_main common) add_check(gtests_libcommon) add_executable(bench_libcommon - bench_logger.cpp + # TODO: need to fix broken src file if necessary + # bench_logger.cpp + bench_mem_utils.cpp ) -target_link_libraries(bench_libcommon benchmark::benchmark_main common) +target_link_libraries(bench_libcommon benchmark::benchmark_main common m) add_executable (dump_variable dump_variable.cpp) target_link_libraries (dump_variable clickhouse_common_io) diff --git a/libs/libcommon/src/tests/bench_mem_utils.cpp b/libs/libcommon/src/tests/bench_mem_utils.cpp new file mode 100644 index 00000000000..7531202ad7f --- /dev/null +++ b/libs/libcommon/src/tests/bench_mem_utils.cpp @@ -0,0 +1,206 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +#include +#include +#include +#include +#include +#include +#include + + +namespace bench +{ + +constexpr size_t RESERVE_OFFSET = 200; +constexpr size_t TEST_ALIGN_SIZE = 64; +static_assert(RESERVE_OFFSET > TEST_ALIGN_SIZE * 2); +constexpr char DEFAULT_INIT_CHAR = '0'; +static constexpr size_t TEST_ALIGN_OFF_1 = 15; +static constexpr size_t TEST_ALIGN_OFF_2 = 31; + +static_assert(TEST_ALIGN_SIZE > TEST_ALIGN_OFF_1); +static_assert(TEST_ALIGN_SIZE > TEST_ALIGN_OFF_2); + +static constexpr bool varify_res = false; + +template +class MemUtilsEqual : public benchmark::Fixture +{ +protected: + std::string inner_data1; + std::string inner_data2; + std::string_view data1; + std::string_view data2; + +public: + static constexpr size_t max_size = max_src_size; + + void SetUp(const ::benchmark::State & /*state*/) override + { + inner_data1.resize(max_size + RESERVE_OFFSET, DEFAULT_INIT_CHAR); + inner_data2 = inner_data1; + + { + const auto * src = reinterpret_cast((size_t(inner_data1.data()) + TEST_ALIGN_SIZE - 1) / TEST_ALIGN_SIZE * TEST_ALIGN_SIZE + TEST_ALIGN_OFF_1); // start address not aligned + size_t size = inner_data1.data() + inner_data1.size() - src; + data1 = {src, size}; + } + + { + const auto * src = reinterpret_cast((size_t(inner_data2.data()) + TEST_ALIGN_SIZE - 1) / TEST_ALIGN_SIZE * TEST_ALIGN_SIZE + TEST_ALIGN_OFF_2); // start address not aligned + size_t size = inner_data2.data() + inner_data2.size() - src; + data2 = {src, size}; + } + } +}; + +template +class MemUtilsStrStr : public benchmark::Fixture +{ +protected: + std::vector inner_data1; + std::vector inner_data2; + + std::vector data1; + std::vector data2; + +public: + static constexpr int check_char = -1; + + void SetUp(const ::benchmark::State & /*state*/) override + { + inner_data1.resize(max_cnt); + inner_data2.resize(max_cnt); + data1.resize(max_cnt); + data2.resize(max_cnt); + + for (size_t i = 0; i < max_cnt; ++i) + { + { + auto & inner_data = inner_data1[i]; + inner_data.resize(max_src_size + RESERVE_OFFSET, DEFAULT_INIT_CHAR); + auto * src = reinterpret_cast((size_t(inner_data.data()) + TEST_ALIGN_SIZE - 1) / TEST_ALIGN_SIZE * TEST_ALIGN_SIZE + TEST_ALIGN_OFF_1); // start address not aligned + size_t size = max_src_size; + data1[i] = {src, size}; + + char * bg = src + size - max_needle_size; + if (max_needle_size > 1 && size / 2 + 1 != size - max_needle_size) + src[size / 2] = check_char; //set one char + memset(bg, check_char, max_needle_size); + } + { + auto & inner_data = inner_data2[i]; + inner_data.resize(max_needle_size + RESERVE_OFFSET, check_char); + auto * src = reinterpret_cast((size_t(inner_data.data()) + TEST_ALIGN_SIZE - 1) / TEST_ALIGN_SIZE * TEST_ALIGN_SIZE + TEST_ALIGN_OFF_2); // start address not aligned + size_t size = max_needle_size; + data2[i] = {src, size}; + } + } + } +}; + + +ALWAYS_INLINE static inline bool stl_mem_eq(const char * p1, const char * p2, size_t n) +{ + return std::memcmp(p1, p2, n) == 0; // call bcmp@plt +} + +NO_INLINE size_t stl_str_find(std::string_view s, std::string_view p) +{ + return s.find(p); // call memchr@plt -> bcmp@plt +} + +// volatile value is used to prevent compiler optimization for fixed context + +#define BENCH_MEM_EQ(name1, name2, func, iter_cnt) \ + BENCHMARK_DEFINE_F(name1, name2) \ + (benchmark::State & state) \ + { \ + [[maybe_unused]] volatile size_t _volatile_flags = 1; \ + [[maybe_unused]] volatile size_t cnt = max_size; \ + for (auto _ : state) \ + { \ + _volatile_flags = func(data1.data(), data2.data(), cnt); \ + if constexpr (varify_res) \ + { \ + if (unlikely(!_volatile_flags)) \ + exit(-1); \ + } \ + } \ + } \ + BENCHMARK_REGISTER_F(name1, name2)->Iterations(iter_cnt); + + +#define BENCH_MEM_STRSTR(name1, name2, func, iter_cnt) \ + BENCHMARK_DEFINE_F(name1, name2) \ + (benchmark::State & state) \ + { \ + [[maybe_unused]] volatile size_t _volatile_flags = 0; \ + for (auto _ : state) \ + { \ + for (size_t i = 0; i < data1.size(); ++i) \ + { \ + _volatile_flags = func(data1[i], data2[i]); \ + if constexpr (varify_res) \ + { \ + if (_volatile_flags != data1[i].size() - data2[i].size()) \ + { \ + exit(-1); \ + } \ + } \ + } \ + } \ + } \ + BENCHMARK_REGISTER_F(name1, name2)->Iterations(iter_cnt); + + +#define BENCH_MEM_EQ_ALL(max_src_size, iter_cnt) \ + using MemUtilsEqual##_##max_src_size = MemUtilsEqual; \ + BENCH_MEM_EQ(MemUtilsEqual##_##max_src_size, stl_mem_eq, stl_mem_eq, iter_cnt) \ + BENCH_MEM_EQ(MemUtilsEqual##_##max_src_size, mem_utils_memoryEqual_avx512, mem_utils::memoryEqual, iter_cnt) \ + BENCH_MEM_EQ(MemUtilsEqual##_##max_src_size, avx2_mem_equal, mem_utils::avx2_mem_equal, iter_cnt) + +#define BENCH_MEM_STRSTR_ALL(max_cnt, max_src_size, max_needle_size, iter_cnt) \ + using MemUtilsStrStr##_##max_src_size##_##max_needle_size = MemUtilsStrStr; \ + BENCH_MEM_STRSTR(MemUtilsStrStr##_##max_src_size##_##max_needle_size, stl_str_find, stl_str_find, iter_cnt) \ + BENCH_MEM_STRSTR(MemUtilsStrStr##_##max_src_size##_##max_needle_size, avx2_strstr, mem_utils::avx2_strstr, iter_cnt) + +BENCH_MEM_EQ_ALL(13, 2000) +BENCH_MEM_EQ_ALL(65, 2000) +BENCH_MEM_EQ_ALL(100, 500) +BENCH_MEM_EQ_ALL(10000, 500) +BENCH_MEM_EQ_ALL(100000, 500) +BENCH_MEM_EQ_ALL(1000000, 200) + +BENCH_MEM_STRSTR_ALL(512, 1024, 1, 100); +BENCH_MEM_STRSTR_ALL(512, 1024, 7, 100); +BENCH_MEM_STRSTR_ALL(512, 1024, 15, 100); +BENCH_MEM_STRSTR_ALL(512, 1024, 31, 100); +BENCH_MEM_STRSTR_ALL(512, 1024, 63, 100); + +BENCH_MEM_STRSTR_ALL(512, 80, 1, 100); +BENCH_MEM_STRSTR_ALL(512, 80, 7, 100); +BENCH_MEM_STRSTR_ALL(512, 80, 15, 100); +BENCH_MEM_STRSTR_ALL(512, 80, 31, 100); + +} // namespace bench diff --git a/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp b/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp new file mode 100644 index 00000000000..3895577dca5 --- /dev/null +++ b/libs/libcommon/src/tests/gtest_mem_utils_opt.cpp @@ -0,0 +1,174 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +#include +#include +#include +#include +#include + +#if defined(TIFLASH_ENABLE_AVX_SUPPORT) + +void TestFunc(size_t size) +{ + std::string oa(size + 2, '0'); + oa[size] = char(1); + std::string ob = oa; + ob[size] = char(2); + + std::string_view a{oa.data(), size}; + std::string_view b{ob.data(), size}; + ASSERT_TRUE(mem_utils::IsStrViewEqual(a, b)); + ASSERT_EQ(nullptr, mem_utils::avx2_memchr(a.data(), size, '1')); + + for (size_t first_fail_pos = 0; first_fail_pos < size; ++first_fail_pos) + { + auto tmp = ob[first_fail_pos]; + ob[first_fail_pos] = '1'; + ASSERT_EQ(first_fail_pos, mem_utils::StrFind(b, "1")); + ASSERT_FALSE(mem_utils::IsStrViewEqual(a, b)); + ASSERT_TRUE(mem_utils::IsStrViewEqual({a.data(), first_fail_pos}, {b.data(), first_fail_pos})); + ob[first_fail_pos] = tmp; + } +} + +void TestStrCmpFunc(size_t size) +{ + std::string oa(size + 2, '1'); + oa[size] = char(1); + std::string ob = oa; + ob[size] = char(2); + + std::string_view a{oa.data(), size}; + std::string_view b{ob.data(), size}; + + ASSERT_EQ(mem_utils::CompareStrView(a, b), 0); + + for (size_t first_fail_pos = 0; first_fail_pos < size; ++first_fail_pos) + { + auto tmp = ob[first_fail_pos]; + ob[first_fail_pos] = '2'; + ASSERT_LT(mem_utils::CompareStrView(a, b), 0); + ASSERT_GT(mem_utils::CompareStrView(b, a), 0); + ob[first_fail_pos] = '0'; + ASSERT_GT(mem_utils::CompareStrView(a, b), 0); + ASSERT_LT(mem_utils::CompareStrView(b, a), 0); + ob[first_fail_pos] = tmp; + } +} + +TEST(MemUtilsTestOPT, CompareNormal) +{ + // size not equal + ASSERT_FALSE(mem_utils::IsStrViewEqual("123", "1")); + + for (size_t size = 0; size < (256 + 128 + 10); ++size) + { + TestFunc(size); + } + { + std::string_view a = "12"; + std::string_view b = "1234"; + ASSERT_EQ(mem_utils::StrFind(a, b), -1); + ASSERT_EQ(*mem_utils::avx2_memchr(b.data(), b.size(), '4'), '4'); + + a = ""; + b = ""; + ASSERT_EQ(mem_utils::StrFind(a, b), 0); + } + for (size_t size = 0; size < 256; ++size) + { + std::string a(size + 50, char(0)); + auto * start = reinterpret_cast((size_t(a.data()) + 32 - 1) / 32 * 32 + 10); // start address not aligned + + for (size_t first_pos = 0; first_pos < size; ++first_pos) + { + for (size_t needle_size = 1; needle_size + first_pos <= size; ++needle_size) + { + std::memset(start + first_pos, -1, needle_size); + { + ASSERT_EQ(mem_utils::StrFind({start, size}, {start + first_pos, needle_size}), first_pos); + } + std::memset(start + first_pos, 0, needle_size); + } + } + } + { + size_t size = 10; + std::string a(size + 50, char(0)); + auto * start = reinterpret_cast((size_t(a.data()) + 32 - 1) / 32 * 32 + 10); // start address not aligned + start[-5] = 1; + start[5] = 1; + start[15] = 1; + std::string b(2, char(1)); + ASSERT_EQ(-1, + mem_utils::StrFind({start, size}, b)); + } + { + size_t size = 32 - 10 + 6; + std::string a(size + 50, char(0)); + auto * start = reinterpret_cast((size_t(a.data()) + 32 - 1) / 32 * 32 + 10); // start address not aligned + start[-5] = 1; + start[23] = 1; + start[29] = 1; + std::string b(2, char(1)); + ASSERT_EQ(-1, + mem_utils::StrFind({start, size}, b)); + } + { + size_t size = 32 - 10 + 32 + 5; + std::string a(size + 50, char(0)); + auto * start = reinterpret_cast((size_t(a.data()) + 32 - 1) / 32 * 32 + 10); // start address not aligned + start[23] = 1; + start[23 + 4] = 1; + std::string b(2, char(1)); + ASSERT_EQ(-1, + mem_utils::StrFind({start, size}, b)); + } + { + size_t size = 32 - 10 + 32 * 5 + 5; + std::string a(size + 50, char(0)); + auto * start = reinterpret_cast((size_t(a.data()) + 32 - 1) / 32 * 32 + 10); // start address not aligned + start[22 + 2 * 32] = 1; + start[22 + 2 * 32 + 6] = 1; + std::string b(2, char(1)); + ASSERT_EQ(-1, + mem_utils::StrFind({start, size}, b)); + } +} + +TEST(MemUtilsTestOPT, CompareStr) +{ + // size not equal + ASSERT_EQ(mem_utils::CompareStrView("123", "1"), 1); + ASSERT_EQ(mem_utils::CompareStrView("123", "123"), 0); + ASSERT_EQ(mem_utils::CompareStrView("123", "1234"), -1); + ASSERT_EQ(mem_utils::CompareStrView("1", ""), 1); + ASSERT_EQ(mem_utils::CompareStrView("", ""), 0); + ASSERT_EQ(mem_utils::CompareStrView("", "1"), -1); + + for (size_t size = 0; size < (256 + 128 + 10); ++size) + { + TestStrCmpFunc(size); + } +} + +#endif diff --git a/tests/fullstack-test/expr/from_unixtime.test b/tests/fullstack-test/expr/from_unixtime.test index fdc65a208cd..468dbd2d578 100644 --- a/tests/fullstack-test/expr/from_unixtime.test +++ b/tests/fullstack-test/expr/from_unixtime.test @@ -20,6 +20,6 @@ func> wait_table test t # This will throw error for now, but before https://github.com/pingcap/tics/issues/1433 fixed, it will cause segmentation fault mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t where FROM_UNIXTIME(1447430881, a) -ERROR 1105 (HY000) at line 1: Argument at index 1 for function fromUnixTime must be constant +{#REGEXP}.*Argument at index 1 for function fromUnixTime must be constant.* mysql> drop table if exists test.t diff --git a/tests/fullstack-test/expr/str_to_date.test b/tests/fullstack-test/expr/str_to_date.test index 119e7609815..b9d12159315 100644 --- a/tests/fullstack-test/expr/str_to_date.test +++ b/tests/fullstack-test/expr/str_to_date.test @@ -39,15 +39,15 @@ mysql> set sql_mode=''; set tidb_allow_mpp=1; set tidb_isolation_read_engines='t ## Test suite 1 - Disallow zero day # The sql_mode does not effect the result set #mysql> set sql_mode='NO_ZERO_IN_DATE,NO_ZERO_DATE'; set tidb_allow_mpp=1; set tidb_isolation_read_engines='tiflash'; explain select a, ifnull(str_to_date(a, '%d/%m/%Y'),str_to_date('00/00/0000', '%d/%m/%Y')) as date, count(*) as cnt from test.t where suite = 1 group by a,date order by a -mysql> set sql_mode='NO_ZERO_IN_DATE,NO_ZERO_DATE'; set tidb_allow_mpp=1; set tidb_isolation_read_engines='tiflash'; select a, ifnull(str_to_date(a, '%d/%m/%Y'),str_to_date('00/00/0000', '%d/%m/%Y')) as date, count(*) as cnt from test.t where suite = 1 group by a,date order by a -+------------+------------+-----+ -| a | date | cnt | -+------------+------------+-----+ -| 0/0/2012 | NULL | 1 | -| 00/00/0000 | NULL | 1 | -| 13/05/2019 | 2019-05-13 | 1 | -| abc | NULL | 1 | -+------------+------------+-----+ +#mysql> set sql_mode='NO_ZERO_IN_DATE,NO_ZERO_DATE'; set tidb_allow_mpp=1; set tidb_isolation_read_engines='tiflash'; select a, ifnull(str_to_date(a, '%d/%m/%Y'),str_to_date('00/00/0000', '%d/%m/%Y')) as date, count(*) as cnt from test.t where suite = 1 group by a,date order by a +#+------------+------------+-----+ +#| a | date | cnt | +#+------------+------------+-----+ +#| 0/0/2012 | NULL | 1 | +#| 00/00/0000 | NULL | 1 | +#| 13/05/2019 | 2019-05-13 | 1 | +#| abc | NULL | 1 | +#+------------+------------+-----+ ## Test suite 2 - showing datetime with fractions #mysql> set sql_mode=''; set tidb_allow_mpp=1; set tidb_isolation_read_engines='tiflash'; explain select a, str_to_date(a, '%d/%b/%Y %H:%i:%S.%f') as date from test.t where suite = 2 group by a,date order by a diff --git a/tests/fullstack-test2/variables/set_variable_fastscan.test b/tests/fullstack-test2/variables/set_variable_fastscan.test new file mode 100644 index 00000000000..dbd62bc53ce --- /dev/null +++ b/tests/fullstack-test2/variables/set_variable_fastscan.test @@ -0,0 +1,93 @@ +# Copyright 2022 PingCAP, Ltd. +# +# 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. + +## test correctness in simple case +mysql> drop table if exists test.t +mysql> create table test.t (a int, b int) +mysql> alter table test.t set tiflash replica 1 + +func> wait_table test t + +mysql> insert into test.t values(1,2); +mysql> insert into test.t values(2,4); +mysql> insert into test.t values(5,7); + +mysql> set session tiflash_fastscan=ON; set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 1 | 2| +| 2 | 4| +| 5 | 7| ++------+-----+ + +mysql> delete from test.t where a = 1; + +mysql> set session tiflash_fastscan=ON; set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 1 | 2| +| 2 | 4| +| 5 | 7| ++------+-----+ + +## verify session variable tiflash_fastscan will not effect global variable +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 2 | 4| +| 5 | 7| ++------+-----+ + +mysql> set session tiflash_fastscan=OFF; set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 2 | 4| +| 5 | 7| ++------+-----+ + +mysql> alter table test.t compact tiflash replica; + +mysql> set session tiflash_fastscan=ON; set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 1 | 2| +| 2 | 4| +| 5 | 7| ++------+-----+ + +## verfiy set global session tiflash_fastscan will not make other current session take effect. +mysql> set global tiflash_fastscan=ON; set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 2 | 4| +| 5 | 7| ++------+-----+ + +## verfiy set global session tiflash_fastscan will make other new session take effect. +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t; ++------+-----+ +| a | b | ++------+-----+ +| 1 | 2| +| 2 | 4| +| 5 | 7| ++------+-----+ + +mysql> set global tiflash_fastscan=OFF; \ No newline at end of file