diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 6ea4d608b1c..0080c6866d8 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 6ea4d608b1c03fab89d17f54a2e399602231e27c +Subproject commit 0080c6866d8bbba32bdf198437dbc98cc078ea7e diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 27283c0f24a..da071507a72 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -233,7 +233,8 @@ class ColumnConst final : public COWPtrHelper template T getValue() const { - return getField().safeGet::Type>(); + auto && tmp = getField(); + return std::move(tmp.safeGet::Type>()); } }; diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index b7ce9fd1e89..8673784c590 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -22,6 +22,8 @@ M(OpenFileForReadWrite) \ M(MemoryTracking) \ M(MemoryTrackingInBackgroundProcessingPool) \ + M(LogicalCPUCores) \ + M(MemoryCapacity) \ M(PSMVCCNumSnapshots) \ M(PSMVCCSnapshotsList) \ M(RWLockWaitingReaders) \ diff --git a/dbms/src/Common/MPMCQueue.h b/dbms/src/Common/MPMCQueue.h index f550ecc7ca2..31dfc65a174 100644 --- a/dbms/src/Common/MPMCQueue.h +++ b/dbms/src/Common/MPMCQueue.h @@ -74,56 +74,80 @@ class MPMCQueue destruct(getObj(read_pos)); } - /// Block util: + /// Block until: /// 1. Pop succeeds with a valid T: return true. /// 2. The queue is cancelled or finished: return false. - bool pop(T & obj) + ALWAYS_INLINE bool pop(T & obj) { - return popObj(obj); + return popObj(obj); } - /// Besides all conditions mentioned at `pop`, `tryPop` will return false if `timeout` is exceeded. + /// Besides all conditions mentioned at `pop`, `popTimeout` will return false if `timeout` is exceeded. template - bool tryPop(T & obj, const Duration & timeout) + ALWAYS_INLINE bool popTimeout(T & obj, const Duration & timeout) { /// std::condition_variable::wait_until will always use system_clock. auto deadline = std::chrono::system_clock::now() + timeout; - return popObj(obj, &deadline); + return popObj(obj, &deadline); } - /// Block util: + /// Non-blocking function. + /// Return true if pop succeed. + /// else return false. + ALWAYS_INLINE bool tryPop(T & obj) + { + return popObj(obj); + } + + /// Block until: /// 1. Push succeeds and return true. /// 2. The queue is cancelled and return false. /// 3. The queue has finished and return false. template ALWAYS_INLINE bool push(U && u) { - return pushObj(std::forward(u)); + return pushObj(std::forward(u)); } - /// Besides all conditions mentioned at `push`, `tryPush` will return false if `timeout` is exceeded. + /// Besides all conditions mentioned at `push`, `pushTimeout` will return false if `timeout` is exceeded. template - ALWAYS_INLINE bool tryPush(U && u, const Duration & timeout) + ALWAYS_INLINE bool pushTimeout(U && u, const Duration & timeout) { /// std::condition_variable::wait_until will always use system_clock. auto deadline = std::chrono::system_clock::now() + timeout; - return pushObj(std::forward(u), &deadline); + return pushObj(std::forward(u), &deadline); + } + + /// Non-blocking function. + /// Return true if push succeed. + /// else return false. + template + ALWAYS_INLINE bool tryPush(U && u) + { + return pushObj(std::forward(u)); } /// The same as `push` except it will construct the object in place. template ALWAYS_INLINE bool emplace(Args &&... args) { - return emplaceObj(nullptr, std::forward(args)...); + return emplaceObj(nullptr, std::forward(args)...); } - /// The same as `tryPush` except it will construct the object in place. + /// The same as `pushTimeout` except it will construct the object in place. template - ALWAYS_INLINE bool tryEmplace(Args &&... args, const Duration & timeout) + ALWAYS_INLINE bool emplaceTimeout(Args &&... args, const Duration & timeout) { /// std::condition_variable::wait_until will always use system_clock. auto deadline = std::chrono::system_clock::now() + timeout; - return emplaceObj(&deadline, std::forward(args)...); + return emplaceObj(&deadline, std::forward(args)...); + } + + /// The same as `tryPush` except it will construct the object in place. + template + ALWAYS_INLINE bool tryEmplace(Args &&... args) + { + return emplaceObj(nullptr, std::forward(args)...); } /// Cancel a NORMAL queue will wake up all blocking readers and writers. @@ -233,7 +257,8 @@ class MPMCQueue } } - bool popObj(T & res, const TimePoint * deadline = nullptr) + template + bool popObj(T & res, [[maybe_unused]] const TimePoint * deadline = nullptr) { #ifdef __APPLE__ WaitingNode node; @@ -241,14 +266,16 @@ class MPMCQueue thread_local WaitingNode node; #endif { - /// read_pos < write_pos means the queue isn't empty - auto pred = [&] { - return read_pos < write_pos || !isNormal(); - }; - std::unique_lock lock(mu); - wait(lock, reader_head, node, pred, deadline); + if constexpr (need_wait) + { + /// read_pos < write_pos means the queue isn't empty + auto pred = [&] { + return read_pos < write_pos || !isNormal(); + }; + wait(lock, reader_head, node, pred, deadline); + } if (!isCancelled() && read_pos < write_pos) { @@ -272,21 +299,23 @@ class MPMCQueue return false; } - template - bool assignObj(const TimePoint * deadline, F && assigner) + template + bool assignObj([[maybe_unused]] const TimePoint * deadline, F && assigner) { #ifdef __APPLE__ WaitingNode node; #else thread_local WaitingNode node; #endif - auto pred = [&] { - return write_pos - read_pos < capacity || !isNormal(); - }; - std::unique_lock lock(mu); - wait(lock, writer_head, node, pred, deadline); + if constexpr (need_wait) + { + auto pred = [&] { + return write_pos - read_pos < capacity || !isNormal(); + }; + wait(lock, writer_head, node, pred, deadline); + } /// double check status after potential wait /// check write_pos because timeouted will also reach here. @@ -305,16 +334,16 @@ class MPMCQueue return false; } - template + template ALWAYS_INLINE bool pushObj(U && u, const TimePoint * deadline = nullptr) { - return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(u)); }); + return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(u)); }); } - template + template ALWAYS_INLINE bool emplaceObj(const TimePoint * deadline, Args &&... args) { - return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(args)...); }); + return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(args)...); }); } ALWAYS_INLINE bool isNormal() const diff --git a/dbms/src/Common/MyDuration.cpp b/dbms/src/Common/MyDuration.cpp index 8801ae0de44..513c40b6dbc 100644 --- a/dbms/src/Common/MyDuration.cpp +++ b/dbms/src/Common/MyDuration.cpp @@ -67,4 +67,4 @@ String MyDuration::toString() const auto frac_str = fmt::format("{:06}", microsecond); return fmt::format(fmt_str, sign > 0 ? "" : "-", hour, minute, second, frac_str); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Common/tests/gtest_mpmc_queue.cpp b/dbms/src/Common/tests/gtest_mpmc_queue.cpp index 85ad1892067..3f2748b452b 100644 --- a/dbms/src/Common/tests/gtest_mpmc_queue.cpp +++ b/dbms/src/Common/tests/gtest_mpmc_queue.cpp @@ -98,12 +98,14 @@ class MPMCQueueTest : public ::testing::Test void testCannotTryPush(MPMCQueue & queue) { auto old_size = queue.size(); - auto res = queue.tryPush(ValueHelper::make(-1), std::chrono::microseconds(1)); - auto new_size = queue.size(); - if (res) + bool ok1 = queue.tryPush(ValueHelper::make(-1)); + auto new_size1 = queue.size(); + bool ok2 = queue.pushTimeout(ValueHelper::make(-1), std::chrono::microseconds(1)); + auto new_size2 = queue.size(); + if (ok1 || ok2) throw TiFlashTestException("Should push fail"); - if (old_size != new_size) - throw TiFlashTestException(fmt::format("Size changed from {} to {} without push", old_size, new_size)); + if (old_size != new_size1 || old_size != new_size2) + throw TiFlashTestException(fmt::format("Size changed from {} to {} and {} without push", old_size, new_size1, new_size2)); } template @@ -124,12 +126,14 @@ class MPMCQueueTest : public ::testing::Test { auto old_size = queue.size(); T res; - bool ok = queue.tryPop(res, std::chrono::microseconds(1)); - auto new_size = queue.size(); - if (ok) + bool ok1 = queue.tryPop(res); + auto new_size1 = queue.size(); + bool ok2 = queue.popTimeout(res, std::chrono::microseconds(1)); + auto new_size2 = queue.size(); + if (ok1 || ok2) throw TiFlashTestException("Should pop fail"); - if (old_size != new_size) - throw TiFlashTestException(fmt::format("Size changed from {} to {} without pop", old_size, new_size)); + if (old_size != new_size1 || old_size != new_size2) + throw TiFlashTestException(fmt::format("Size changed from {} to {} and {} without pop", old_size, new_size1, new_size2)); } template @@ -474,7 +478,6 @@ class MPMCQueueTest : public ::testing::Test throwOrMove(std::move(rhs)); } - ThrowInjectable & operator=(ThrowInjectable && rhs) { if (this != &rhs) diff --git a/dbms/src/Common/tests/mpmc_queue_perftest.cpp b/dbms/src/Common/tests/mpmc_queue_perftest.cpp index d047b5d498f..ba0d00001a3 100644 --- a/dbms/src/Common/tests/mpmc_queue_perftest.cpp +++ b/dbms/src/Common/tests/mpmc_queue_perftest.cpp @@ -87,7 +87,7 @@ struct Helper> template static void pushOneTo(MPMCQueue & queue, U && data) { - queue.tryPush(std::forward(data), std::chrono::milliseconds(1)); + queue.pushTimeout(std::forward(data), std::chrono::milliseconds(1)); } }; diff --git a/dbms/src/DataTypes/NumberTraits.h b/dbms/src/DataTypes/NumberTraits.h index 925628a8894..a8b91b88075 100644 --- a/dbms/src/DataTypes/NumberTraits.h +++ b/dbms/src/DataTypes/NumberTraits.h @@ -277,6 +277,7 @@ struct ResultOfAbs> }; /** For bitwise operations, an integer is obtained with number of bits is equal to the maximum of the arguments. + * todo: note that MySQL handles only unsigned 64-bit integer argument and result values. We should refine the code. */ template struct ResultOfBit diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index 3f633c08e67..df993d8e6e9 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -118,6 +118,10 @@ DBGInvoker::DBGInvoker() regSchemalessFunc("mapped_database", dbgFuncMappedDatabase); regSchemalessFunc("mapped_table", dbgFuncMappedTable); 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/dbgFuncSchema.cpp b/dbms/src/Debug/dbgFuncSchema.cpp index c388015dc10..9ef07f16e8b 100644 --- a/dbms/src/Debug/dbgFuncSchema.cpp +++ b/dbms/src/Debug/dbgFuncSchema.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -137,4 +138,5 @@ void dbgFuncIsTombstone(Context & context, const ASTs & args, DBGInvoker::Printe output(fmt_buf.toString()); } + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Debug/dbgFuncSchema.h b/dbms/src/Debug/dbgFuncSchema.h index 162bc0af46b..51ab3ad41cf 100644 --- a/dbms/src/Debug/dbgFuncSchema.h +++ b/dbms/src/Debug/dbgFuncSchema.h @@ -46,5 +46,4 @@ void dbgFuncResetSchemas(Context & context, const ASTs & args, DBGInvoker::Print // Usage: // ./storage-client.sh "DBGInvoke is_tombstone(db_name, table_name)" void dbgFuncIsTombstone(Context & context, const ASTs & args, DBGInvoker::Printer output); - } // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchemaName.cpp b/dbms/src/Debug/dbgFuncSchemaName.cpp index 4c2ad86bd62..3aa7b6e3af4 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.cpp +++ b/dbms/src/Debug/dbgFuncSchemaName.cpp @@ -128,4 +128,109 @@ BlockInputStreamPtr dbgFuncQueryMapped(Context & context, const ASTs & args) return executeQuery(query, context, true).in; } + +void dbgFuncGetTiflashReplicaCount(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((std::to_string(managed_storage->getTableInfo().replica_info.count))); + + output(fmt_buf.toString()); +} + +void dbgFuncGetPartitionTablesTiflashReplicaCount(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((std::to_string(partition_storage->getTableInfo().replica_info.count))); + fmt_buf.append("/"); + } + + 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 8e95aaab908..ec18f89e911 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.h +++ b/dbms/src/Debug/dbgFuncSchemaName.h @@ -40,4 +40,24 @@ void dbgFuncMappedTable(Context & context, const ASTs & args, DBGInvoker::Printe // ./storage-client.sh "DBGInvoke query_mapped('select * from $d.$t', database_name[, table_name])" BlockInputStreamPtr dbgFuncQueryMapped(Context & context, const ASTs & args); +// Get table's tiflash replica counts with mapped table name +// Usage: +// ./storage-client.sh "DBGInvoke get_tiflash_replica_count(db_name, table_name)" +void dbgFuncGetTiflashReplicaCount(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_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/DAGExpressionAnalyzerHelper.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp index ee529680d28..23bbb4586b3 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp @@ -450,6 +450,7 @@ DAGExpressionAnalyzerHelper::FunctionBuilderMap DAGExpressionAnalyzerHelper::fun {"bitOr", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, {"bitXor", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, {"bitNot", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, + {"bitShiftRight", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, {"leftUTF8", DAGExpressionAnalyzerHelper::buildLeftUTF8Function}, {"date_add", DAGExpressionAnalyzerHelper::buildDateAddOrSubFunction}, {"date_sub", DAGExpressionAnalyzerHelper::buildDateAddOrSubFunction}, diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 9ffa29cd14d..a4d491e5637 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -332,7 +332,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::DecimalIsFalseWithNull, "isFalseWithNull"}, //{tipb::ScalarFuncSig::LeftShift, "cast"}, - //{tipb::ScalarFuncSig::RightShift, "cast"}, + {tipb::ScalarFuncSig::RightShift, "bitShiftRight"}, //{tipb::ScalarFuncSig::BitCount, "cast"}, //{tipb::ScalarFuncSig::GetParamString, "cast"}, @@ -561,7 +561,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::Quarter, "toQuarter"}, //{tipb::ScalarFuncSig::SecToTime, "cast"}, - //{tipb::ScalarFuncSig::TimeToSec, "cast"}, + {tipb::ScalarFuncSig::TimeToSec, "tidbTimeToSec"}, //{tipb::ScalarFuncSig::TimestampAdd, "cast"}, {tipb::ScalarFuncSig::ToDays, "tidbToDays"}, {tipb::ScalarFuncSig::ToSeconds, "tidbToSeconds"}, @@ -648,8 +648,8 @@ const std::unordered_map scalar_func_map({ //{tipb::ScalarFuncSig::Quote, "cast"}, //{tipb::ScalarFuncSig::Repeat, "cast"}, {tipb::ScalarFuncSig::Replace, "replaceAll"}, - //{tipb::ScalarFuncSig::ReverseUTF8, "cast"}, - //{tipb::ScalarFuncSig::Reverse, "cast"}, + {tipb::ScalarFuncSig::ReverseUTF8, "reverseUTF8"}, + {tipb::ScalarFuncSig::Reverse, "reverse"}, {tipb::ScalarFuncSig::RightUTF8, "rightUTF8"}, //{tipb::ScalarFuncSig::Right, "cast"}, {tipb::ScalarFuncSig::RpadUTF8, "rpadUTF8"}, diff --git a/dbms/src/Flash/EstablishCall.cpp b/dbms/src/Flash/EstablishCall.cpp index 89857a2407e..2f8c7c15f56 100644 --- a/dbms/src/Flash/EstablishCall.cpp +++ b/dbms/src/Flash/EstablishCall.cpp @@ -143,7 +143,7 @@ void EstablishCallData::finishTunnelAndResponder() state = FINISH; if (mpp_tunnel) { - mpp_tunnel->consumerFinish("grpc writes failed.", true); //trigger mpp tunnel finish work + mpp_tunnel->consumerFinish(fmt::format("{}: finishTunnelAndResponder called.", mpp_tunnel->id()), true); //trigger mpp tunnel finish work } grpc::Status status(static_cast(GRPC_STATUS_UNKNOWN), "Consumer exits unexpected, grpc writes failed."); responder.Finish(status, this); diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 3b36adf2c40..966babb832f 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -424,7 +424,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn for (Int32 i = 0; i < check_waiting_requests_freq; ++i) { AsyncHandler * handler = nullptr; - if (unlikely(!ready_requests.tryPop(handler, timeout))) + if (unlikely(!ready_requests.popTimeout(handler, timeout))) break; handler->handle(); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c2d5e6f49f8..da8f3034abc 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -379,7 +379,7 @@ void MPPTask::runImpl() } catch (...) { - err_msg = getCurrentExceptionMessage(true); + err_msg = getCurrentExceptionMessage(true, true); } if (err_msg.empty()) @@ -405,6 +405,8 @@ void MPPTask::runImpl() if (status == RUNNING) { LOG_FMT_ERROR(log, "task running meets error: {}", err_msg); + /// trim the stack trace to avoid too many useless information in log + trimStackTrace(err_msg); try { handleError(err_msg); diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index 13a7eaad95e..16fe4ae42cc 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -220,7 +220,11 @@ void MPPTunnelBase::sendJob(bool need_lock) err_msg = "fatal error in sendJob()"; } if (!err_msg.empty()) + { + /// append tunnel id to error message + err_msg = fmt::format("{} meet error: {}", tunnel_id, err_msg); LOG_ERROR(log, err_msg); + } consumerFinish(err_msg, need_lock); if (is_async) writer->writeDone(grpc::Status::OK); diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 477c478eef7..21d89b3cd52 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include @@ -27,4 +28,14 @@ mpp::MPPDataPacket getPacketWithError(String reason) return data; } +void trimStackTrace(String & message) +{ + auto stack_trace_pos = message.find("Stack trace"); + if (stack_trace_pos != String::npos) + { + message.resize(stack_trace_pos); + Poco::trimRightInPlace(message); + } +} + } // namespace DB diff --git a/dbms/src/Flash/Mpp/Utils.h b/dbms/src/Flash/Mpp/Utils.h index 67e2dc3f641..021dc4407d5 100644 --- a/dbms/src/Flash/Mpp/Utils.h +++ b/dbms/src/Flash/Mpp/Utils.h @@ -23,5 +23,6 @@ namespace DB { mpp::MPPDataPacket getPacketWithError(String reason); +void trimStackTrace(String & message); } // namespace DB diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp index 47ce2ee6ee6..706c17ed036 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp @@ -382,7 +382,7 @@ TEST_F(TestMPPTunnelBase, WriteError) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, grpc writes failed."); + GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed."); } } @@ -631,7 +631,7 @@ TEST_F(TestMPPTunnelBase, AsyncWriteError) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, grpc writes failed."); + GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed."); } } diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp index da9df20fdf3..dfdb358c46c 100644 --- a/dbms/src/Flash/tests/bench_window.cpp +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -27,7 +27,7 @@ class WindowFunctionBench : public ExchangeBench // build tipb::Window and tipb::Sort. // select row_number() over w1 from t1 window w1 as (partition by c1, c2, c3 order by c1, c2, c3); ExchangeBench::SetUp(state); - MockColumnInfos columns{ + MockColumnInfoVec columns{ {"c1", TiDB::TP::TypeLongLong}, {"c2", TiDB::TP::TypeString}, {"c3", TiDB::TP::TypeLongLong}, diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index ba7d8fd15ee..2d4d4014780 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -96,7 +96,7 @@ try auto request = context.scan("test_db", "test_table_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) - .project("s1") + .project({"s1"}) .build(context); { String expected = R"( @@ -256,7 +256,7 @@ CreatingSets request = context.receive("sender_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) - .project("s1") + .project({"s1"}) .build(context); { String expected = R"( @@ -278,7 +278,7 @@ Union: request = context.receive("sender_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) - .project("s1") + .project({"s1"}) .exchangeSender(tipb::Broadcast) .build(context); { diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index 2ff0fdff780..8443dedeb49 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -86,11 +86,11 @@ TEST_F(ExecutorProjectionTestRunner, Projection) try { /// Check single column - auto request = buildDAGRequest({col_names[4]}); + auto request = buildDAGRequest({col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc)}); /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[4]}); + request = buildDAGRequest({col_names[0], col_names[4]}); executeWithConcurrency(request, { toNullableVec(col_names[0], col0_sorted_asc), @@ -98,14 +98,14 @@ try }); /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); + request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[0], col0_sorted_asc), toNullableVec(col_names[1], col1_sorted_asc), toNullableVec(col_names[4], col4_sorted_asc)}); /// Check duplicate columns - request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); + request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc), toNullableVec(col_names[4], col4_sorted_asc), @@ -114,7 +114,7 @@ try { /// Check large number of columns const size_t col_num = 100; - MockColumnNamesVec projection_input; + MockColumnNameVec projection_input; ColumnsWithTypeAndName columns; auto expect_column = toNullableVec(col_names[4], col4_sorted_asc); @@ -124,7 +124,7 @@ try columns.push_back(expect_column); } - request = buildDAGRequest(projection_input); + request = buildDAGRequest(projection_input); executeWithConcurrency(request, columns); } } @@ -138,18 +138,18 @@ try /// Test "equal" function /// Data type: TypeString - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 0, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, {}, 1, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -158,23 +158,23 @@ try /// Test "greater" function /// Data type: TypeString - request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 1, {}, 1, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 1, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 0, {}, 0, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -183,18 +183,18 @@ try /// Test "and" function /// Data type: TypeString - request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); + request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, 0, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({0, 0, 0, 0, 0, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 0, {}, 1, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -202,7 +202,7 @@ try /// Test "not" function /// Data type: TypeString - request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); + request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), toNullableVec({1, 1, 1, 1, {}, 1, 1}), @@ -210,7 +210,7 @@ try toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 0, {}, 0, 1}), toNullableVec({{}, 0, 0, 1, 0, 0, 0}), diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp index d5466b5c87d..597ac9f279a 100644 --- a/dbms/src/Flash/tests/gtest_topn_executor.cpp +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -52,7 +52,7 @@ class ExecutorTopNTestRunner : public DB::tests::ExecutorTest return context.scan(db_name, table_name).topN(col_name, is_desc, limit_num).build(context); } - std::shared_ptr buildDAGRequest(const String & table_name, MockOrderByItems order_by_items, int limit, MockAsts func_proj_ast = {}, MockColumnNames out_proj_ast = {}) + std::shared_ptr buildDAGRequest(const String & table_name, MockOrderByItemVec order_by_items, int limit, MockAstVec func_proj_ast = {}, MockColumnNameVec out_proj_ast = {}) { if (func_proj_ast.size() == 0) return context.scan(db_name, table_name).topN(order_by_items, limit).build(context); @@ -126,7 +126,7 @@ try toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "china", "korea", "usa"}), toNullableVec(col_name[3], ColumnWithInt32{-300, {}, {}, 900, 1300, 0})}}; - std::vector order_by_items{ + std::vector order_by_items{ /// select * from clerk order by age DESC, gender DESC; {MockOrderByItem(col_name[0], true), MockOrderByItem(col_name[1], true)}, /// select * from clerk order by gender DESC, salary ASC; @@ -150,9 +150,9 @@ try { std::shared_ptr request; std::vector expect_cols; - MockColumnNames output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; - MockAsts func_projection; // Do function operation for topn - MockOrderByItems order_by_items; + MockColumnNameVec output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; + MockAstVec func_projection; // Do function operation for topn + MockOrderByItemVec order_by_items; ASTPtr col0_ast = col(col_name[0]); ASTPtr col1_ast = col(col_name[1]); ASTPtr col2_ast = col(col_name[2]); diff --git a/dbms/src/Functions/CollationOperatorOptimized.h b/dbms/src/Functions/CollationOperatorOptimized.h new file mode 100644 index 00000000000..395ecc5b9eb --- /dev/null +++ b/dbms/src/Functions/CollationOperatorOptimized.h @@ -0,0 +1,210 @@ +// 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 DB +{ + +template +ALWAYS_INLINE inline int signum(T val) +{ + return (0 < val) - (val < 0); +} + +// Check equality is much faster than other comparison. +// - check size first +// - return 0 if equal else 1 +__attribute__((flatten, always_inline, pure)) inline uint8_t RawStrEqualCompare(const std::string_view & lhs, const std::string_view & rhs) +{ + return StringRef(lhs) == StringRef(rhs) ? 0 : 1; +} + +// Compare str view by memcmp +__attribute__((flatten, always_inline, pure)) inline int RawStrCompare(const std::string_view & v1, const std::string_view & v2) +{ + return signum(v1.compare(v2)); +} + +constexpr char SPACE = ' '; + +// Remove tail space +__attribute__((flatten, always_inline, pure)) inline std::string_view RightTrim(const std::string_view & v) +{ + if (likely(v.empty() || v.back() != SPACE)) + return v; + size_t end = v.find_last_not_of(SPACE); + return end == std::string_view::npos ? std::string_view{} : std::string_view(v.data(), end + 1); +} + +__attribute__((flatten, always_inline, pure)) inline int RtrimStrCompare(const std::string_view & va, const std::string_view & vb) +{ + return RawStrCompare(RightTrim(va), RightTrim(vb)); +} + +// If true, only need to check equal or not. +template +struct IsEqualRelated +{ + static constexpr const bool value = false; +}; + +// For `EqualsOp` and `NotEqualsOp`, value is true. +template +struct IsEqualRelated> +{ + static constexpr const bool value = true; +}; +template +struct IsEqualRelated> +{ + static constexpr const bool value = true; +}; + +// Loop columns and invoke callback for each pair. +template +__attribute__((flatten, always_inline)) inline void LoopTwoColumns( + 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) +{ + for (size_t i = 0; i < size; ++i) + { + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + size_t b_size = StringUtil::sizeAt(b_offsets, i) - 1; + const auto * a_ptr = reinterpret_cast(&a_data[StringUtil::offsetAt(a_offsets, i)]); + const auto * b_ptr = reinterpret_cast(&b_data[StringUtil::offsetAt(b_offsets, i)]); + + func({a_ptr, a_size}, {b_ptr, b_size}, i); + } +} + +// Loop one column and invoke callback for each pair. +template +__attribute__((flatten, always_inline)) inline void LoopOneColumn( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + size_t size, + F && func) +{ + for (size_t i = 0; i < size; ++i) + { + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + const auto * a_ptr = reinterpret_cast(&a_data[StringUtil::offsetAt(a_offsets, i)]); + + func({a_ptr, a_size}, i); + } +} + +// Handle str-column compare str-column. +// - Optimize UTF8_BIN and UTF8MB4_BIN +// - Check if columns do NOT contain tail space +// - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way +template +ALWAYS_INLINE inline bool StringVectorStringVector( + 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) +{ + bool use_optimized_path = false; + + switch (collator->getCollatorId()) + { + case TiDB::ITiDBCollator::UTF8MB4_BIN: + case TiDB::ITiDBCollator::UTF8_BIN: + { + size_t size = a_offsets.size(); + + LoopTwoColumns(a_data, a_offsets, b_data, b_offsets, size, [&c](const std::string_view & va, const std::string_view & vb, size_t i) { + if constexpr (IsEqualRelated::value) + { + c[i] = Op::apply(RawStrEqualCompare(RightTrim(va), RightTrim(vb)), 0); + } + else + { + c[i] = Op::apply(RtrimStrCompare(va, vb), 0); + } + }); + + use_optimized_path = true; + + break; + } + default: + break; + } + return use_optimized_path; +} + +// Handle str-column compare const-str. +// - Optimize UTF8_BIN and UTF8MB4_BIN +// - 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 StringVectorConstant( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & b, + const TiDB::TiDBCollatorPtr & collator, + Result & c) +{ + bool use_optimized_path = false; + + switch (collator->getCollatorId()) + { + case TiDB::ITiDBCollator::UTF8MB4_BIN: + case TiDB::ITiDBCollator::UTF8_BIN: + { + size_t size = a_offsets.size(); + + std::string_view tar_str_view = RightTrim(b); // right trim const-str first + + LoopOneColumn(a_data, a_offsets, size, [&c, &tar_str_view](const std::string_view & view, size_t i) { + if constexpr (IsEqualRelated::value) + { + c[i] = Op::apply(RawStrEqualCompare(RightTrim(view), tar_str_view), 0); + } + else + { + c[i] = Op::apply(RawStrCompare(RightTrim(view), tar_str_view), 0); + } + }); + + use_optimized_path = true; + break; + } + default: + break; + } + return use_optimized_path; +} + +} // namespace DB diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 1c63a286452..8f7502fba85 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -301,6 +302,12 @@ struct StringComparisonWithCollatorImpl const TiDB::TiDBCollatorPtr & collator, PaddedPODArray & c) { + bool optimized_path = StringVectorStringVector(a_data, a_offsets, b_data, b_offsets, collator, c); + if (optimized_path) + { + return; + } + size_t size = a_offsets.size(); for (size_t i = 0; i < size; ++i) @@ -317,10 +324,17 @@ struct StringComparisonWithCollatorImpl static void NO_INLINE stringVectorConstant( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const std::string & b, + const std::string_view & b, const TiDB::TiDBCollatorPtr & collator, PaddedPODArray & c) { + bool optimized_path = StringVectorConstant(a_data, a_offsets, b, collator, c); + + if (optimized_path) + { + return; + } + size_t size = a_offsets.size(); ColumnString::Offset b_size = b.size(); const char * b_data = reinterpret_cast(b.data()); @@ -332,7 +346,7 @@ struct StringComparisonWithCollatorImpl } static void constantStringVector( - const std::string & a, + const std::string_view & a, const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, const TiDB::TiDBCollatorPtr & collator, @@ -342,8 +356,8 @@ struct StringComparisonWithCollatorImpl } static void constantConstant( - const std::string & a, - const std::string & b, + const std::string_view & a, + const std::string_view & b, const TiDB::TiDBCollatorPtr & collator, ResultType & c) { @@ -706,6 +720,25 @@ class FunctionComparison : public IFunction } } + static inline std::string_view genConstStrRef(const ColumnConst * c0_const) + { + std::string_view c0_const_str_ref{}; + if (c0_const) + { + if (const auto * c0_const_string = checkAndGetColumn(&c0_const->getDataColumn()); c0_const_string) + { + c0_const_str_ref = std::string_view(c0_const_string->getDataAt(0)); + } + else if (const auto * c0_const_fixed_string = checkAndGetColumn(&c0_const->getDataColumn()); c0_const_fixed_string) + { + c0_const_str_ref = std::string_view(c0_const_fixed_string->getDataAt(0)); + } + else + throw Exception("Logical error: ColumnConst contains not String nor FixedString column", ErrorCodes::ILLEGAL_COLUMN); + } + return c0_const_str_ref; + } + template bool executeStringWithCollator( Block & block, @@ -720,10 +753,13 @@ class FunctionComparison : public IFunction using ResultType = typename ResultColumnType::value_type; using StringImpl = StringComparisonWithCollatorImpl, ResultType>; + std::string_view c0_const_str_ref = genConstStrRef(c0_const); + std::string_view c1_const_str_ref = genConstStrRef(c1_const); + if (c0_const && c1_const) { ResultType res = 0; - StringImpl::constantConstant(c0_const->getValue(), c1_const->getValue(), collator, res); + StringImpl::constantConstant(c0_const_str_ref, c1_const_str_ref, collator, res); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(c0_const->size(), toField(res)); return true; } @@ -745,12 +781,12 @@ class FunctionComparison : public IFunction StringImpl::stringVectorConstant( c0_string->getChars(), c0_string->getOffsets(), - c1_const->getValue(), + c1_const_str_ref, collator, c_res->getData()); else if (c0_const && c1_string) StringImpl::constantStringVector( - c0_const->getValue(), + c0_const_str_ref, c1_string->getChars(), c1_string->getOffsets(), collator, @@ -770,8 +806,8 @@ class FunctionComparison : public IFunction template bool executeString(Block & block, size_t result, const IColumn * c0, const IColumn * c1) const { - const ColumnString * c0_string = checkAndGetColumn(c0); - const ColumnString * c1_string = checkAndGetColumn(c1); + const auto * c0_string = checkAndGetColumn(c0); + const auto * c1_string = checkAndGetColumn(c1); const ColumnConst * c0_const = checkAndGetColumnConstStringOrFixedString(c0); const ColumnConst * c1_const = checkAndGetColumnConstStringOrFixedString(c1); diff --git a/dbms/src/Functions/FunctionsDuration.cpp b/dbms/src/Functions/FunctionsDuration.cpp index ea7b86ac670..9ccafd2794d 100644 --- a/dbms/src/Functions/FunctionsDuration.cpp +++ b/dbms/src/Functions/FunctionsDuration.cpp @@ -97,6 +97,57 @@ void FunctionDurationSplit::executeImpl(Block & block, const ColumnNumbers ErrorCodes::ILLEGAL_COLUMN); }; +template +DataTypePtr FunctionMyDurationToSec::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + if (!arguments[0].type->isMyTime()) + { + throw Exception( + fmt::format("Illegal type {} of the first argument of function {}", arguments[0].type->getName(), getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + return std::make_shared(); +} + +template +void FunctionMyDurationToSec::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const +{ + const auto * from_type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get()); + if (from_type == nullptr) + { + throw Exception( + fmt::format( + "Illegal column {} of the first argument of function {}", + block.getByPosition(arguments[0]).column->getName(), + name), + ErrorCodes::ILLEGAL_COLUMN); + } + + using FromFieldType = typename DataTypeMyDuration::FieldType; + const auto * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get()); + if (col_from != nullptr) + { + const typename ColumnVector::Container & vec_from = col_from->getData(); + const size_t size = vec_from.size(); + auto col_to = ColumnVector::create(size); + typename ColumnVector::Container & vec_to = col_to->getData(); + + for (size_t i = 0; i < size; ++i) + { + MyDuration val(vec_from[i], from_type->getFsp()); + vec_to[i] = Impl::apply(val); + } + block.getByPosition(result).column = std::move(col_to); + } + else + throw Exception( + fmt::format( + "Illegal column {} of the first argument of function {}", + block.getByPosition(arguments[0]).column->getName(), + name), + ErrorCodes::ILLEGAL_COLUMN); +} + struct DurationSplitHourImpl { static constexpr auto name = "hour"; @@ -133,11 +184,27 @@ struct DurationSplitMicroSecondImpl } }; +struct TiDBTimeToSecTransformerImpl +{ + static constexpr auto name = "tidbTimeToSec"; + static Int64 apply(const MyDuration & val) + { + Int64 sign = 1; + if (val.isNeg()) + { + sign = -1; + } + return sign * (val.hours() * 3600 + val.minutes() * 60 + val.seconds()); + } +}; + using FunctionDurationHour = FunctionDurationSplit; using FunctionDurationMinute = FunctionDurationSplit; using FunctionDurationSecond = FunctionDurationSplit; using FunctionDurationMicroSecond = FunctionDurationSplit; +using FunctionToTiDBTimeToSec = FunctionMyDurationToSec; + void registerFunctionsDuration(FunctionFactory & factory) { factory.registerFunction(); @@ -146,5 +213,7 @@ void registerFunctionsDuration(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + + factory.registerFunction(); } } // namespace DB diff --git a/dbms/src/Functions/FunctionsDuration.h b/dbms/src/Functions/FunctionsDuration.h index 4247cde03ff..5bc54d425f4 100644 --- a/dbms/src/Functions/FunctionsDuration.h +++ b/dbms/src/Functions/FunctionsDuration.h @@ -69,4 +69,23 @@ class FunctionDurationSplit : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override; }; +template +class FunctionMyDurationToSec : public IFunction +{ +public: + static constexpr auto name = Impl::name; + + static FunctionPtr create(const Context &) { return std::make_shared(); }; + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override; +}; + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Functions/bitShiftRight.cpp b/dbms/src/Functions/bitShiftRight.cpp index 961f7459f68..90b365771de 100644 --- a/dbms/src/Functions/bitShiftRight.cpp +++ b/dbms/src/Functions/bitShiftRight.cpp @@ -13,6 +13,9 @@ // limitations under the License. #include +#include + +#include namespace DB { @@ -29,7 +32,18 @@ struct BitShiftRightImpl template static Result apply(A a, B b) { - return static_cast(a) >> static_cast(b); + // It is an undefined behavior for shift operation in c++ that the right operand is negative or greater than + // or equal to the number of digits of the bits in the (promoted) left operand. + // See https://en.cppreference.com/w/cpp/language/operator_arithmetic for details. + if (static_cast(b) >= std::numeric_limits(a))>::digits) + { + return static_cast(0); + } + // Note that we do not consider the case that the right operand is negative, + // since other types will all be cast to uint64 before shift operation + // according to DAGExpressionAnalyzerHelper::buildBitwiseFunction. + // Therefore, we simply suppress clang-tidy checking here. + return static_cast(a) >> static_cast(b); // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) } template static Result apply(A, B, UInt8 &) @@ -87,4 +101,4 @@ void registerFunctionBitShiftRight(FunctionFactory & factory) factory.registerFunction(); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Functions/tests/gtest_bitshiftright.cpp b/dbms/src/Functions/tests/gtest_bitshiftright.cpp new file mode 100644 index 00000000000..a4af6336099 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_bitshiftright.cpp @@ -0,0 +1,273 @@ +// 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 +{ +namespace tests +{ +class TestFunctionBitShiftRight : public DB::tests::FunctionTest +{ +}; + +#define ASSERT_BITSHIFTRIGHT(t1, t2, result) \ + ASSERT_COLUMN_EQ(result, executeFunction("bitShiftRight", {t1, t2})) + +TEST_F(TestFunctionBitShiftRight, Simple) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn>({8}), + createColumn>({2}), + createColumn>({2})); +} +CATCH + +/// Note: Only IntX and UIntX will be received by BitShiftRight, others will be casted by TiDB planner. +/// Note: BitShiftRight will further cast other types to UInt64 before doing shift. +TEST_F(TestFunctionBitShiftRight, TypePromotion) +try +{ + // Type Promotion + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + + // Type Promotion across signed/unsigned + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({0}), createColumn>({18446744073709551615ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({0}), createColumn>({18446744073709551615ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, Nullable) +try +{ + // Non Nullable + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + + // Across Nullable and non-Nullable + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, TypeCastWithConst) +try +{ + /// need test these kinds of columns: + /// 1. ColumnVector + /// 2. ColumnVector + /// 3. ColumnConst + /// 4. ColumnConst, value != null + /// 5. ColumnConst, value = null + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 0, 1, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable + + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createColumn({0, 1, 0, 1}), createColumn({1, 0, 1, 0})); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({1, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createConstColumn(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createConstColumn>(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createColumn({0, 1, 0, 1}), createColumn({1, 0, 1, 0})); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({1, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createConstColumn(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createConstColumn>(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, Boundary) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn({127, 127, -128, -128}), createColumn({0, 7, 0, 7}), createColumn({127, 0, 18446744073709551488ull, 144115188075855871ull})); + ASSERT_BITSHIFTRIGHT(createColumn({127, 127, -128, -128}), createColumn({0, 7, 0, 7}), createColumn({127, 0, 18446744073709551488ull, 144115188075855871ull})); + ASSERT_BITSHIFTRIGHT(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 15, 0, 15}), createColumn({32767, 0, 18446744073709518848ull, 562949953421311ull})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), + createColumn({0, 63, 0, 63, 0, 63, 0, 63, 0, 63}), + createColumn({0, 0, 1, 0, 18446744073709551615ull, 1, INT64_MAX, 0, 9223372036854775808ull, 1})); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, UINT64) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn({0, UINT64_MAX}), + createColumn({63, 63}), + createColumn({0, 1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, UINT64_MAX, std::nullopt}), + createColumn>({63, 63, 63}), + createColumn>({0, 1, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, UINT64_MAX, std::nullopt}), + createColumn({63, 63, 63}), + createColumn>({0, 1, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, UINT64_MAX}), + createColumn>({63, 63}), + createColumn>({0, 1})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), + createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX}), + createColumn({0, 0, 1, 0, 18446744073709551615ull, 0, INT64_MAX, 0, 9223372036854775808ull, 0})); + + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, UINT64_MAX, UINT64_MAX}), + createColumn({0, UINT64_MAX, 0, UINT64_MAX}), + createColumn({0, 0, UINT64_MAX, 0})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, 0, UINT64_MAX, UINT64_MAX, 0, std::nullopt}), + createColumn>({0, UINT64_MAX, 0, UINT64_MAX, std::nullopt, 0}), + createColumn>({0, 0, UINT64_MAX, 0, std::nullopt, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, 0, UINT64_MAX, UINT64_MAX, std::nullopt}), + createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0}), + createColumn>({0, 0, UINT64_MAX, 0, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0}), + createColumn>({0, 0, UINT64_MAX, UINT64_MAX, std::nullopt}), + createColumn>({0, UINT64_MAX, 0, 0, std::nullopt})); + + /* + std::mt19937 gen(std::random_device{}()); + std::uniform_int_distribution dis( + std::numeric_limits::min(), + std::numeric_limits::max() + ); + size_t count = 100; + std::vector v1(count), v2(count), res(count); + for (size_t i=0; i> v2[i]; + } + */ + // clang-format off + ASSERT_BITSHIFTRIGHT(createColumn({4286230172992429668ull,11550684080080434735ull,775195682263841867ull,18390588538388462661ull,15578761645824658314ull,20662948907547635ull,8403266546632871011ull,10316916867086714284ull,14494183568060929367ull,11741337603037632348ull,10803264694948981380ull,2181969932373516503ull,9673801579564730047ull,12998855911221966916ull,13852157931865274857ull,9203926828777338586ull,8903261359104369984ull,3296258311466476456ull,14658801806079697908ull,7542518003247963618ull,7751150277360944372ull,12225694156629117269ull,3173837214287201256ull,10555082060194839563ull,14202570947308501213ull,13841194359225980123ull,9085267378073816945ull,15975493157631073381ull,1890233386459299033ull,2368634323417847398ull,691423931511513606ull,986000479038857169ull,6676906740954304741ull,2841686799872009560ull,6483676442160212821ull,12550114481083571140ull,1973026146580965947ull,15006687639313690830ull,6443617813685195609ull,13648732879238232658ull,173820604016606515ull,2669428687588070677ull,15361476519767969236ull,8957522718906827285ull,10484385204137290737ull,12390466571993898199ull,13655746682011856065ull,4183302523705398003ull,9898692767945122925ull,16701902679050716746ull,15003324714492513897ull,15554724240808081962ull,7754458312088240871ull,16060968032680196798ull,12619581440986221928ull,15462661961676206824ull,2991773628650321635ull,16341599119345297909ull,14943939970889580769ull,17589764776976679210ull,15274914527536421890ull,16268454608136611433ull,14617646699124891378ull,466927094873143934ull,10558583305251737283ull,255559140356160501ull,5962789691899784330ull,8004603198837555992ull,1881892337023478820ull,6549167700870881840ull,17551996157828573642ull,3349744237253314638ull,2876698686583880568ull,16792783373922568330ull,16231348759981899800ull,17731631990557975899ull,1305376485657663531ull,3568754485566225727ull,10076204423028931225ull,1206238310176455071ull,4297062324543635867ull,5116785256928623516ull,4216305034157620433ull,412817651268481791ull,11256299741838589766ull,10786197076871163667ull,8588357635228913652ull,6361409982074778071ull,4750871994764527580ull,12851835128796581697ull,13871712051825681122ull,12445309465661589227ull,1668617678034382020ull,10152918068481134781ull,16242941973571224246ull,12988338226657152812ull,2352083670492692674ull,10735026236980245779ull,14986388012066843516ull,17651064432466444102ull}), + createColumn({0,58,55,24,5,35,34,54,43,45,17,36,51,54,19,55,55,8,37,49,15,11,36,0,5,41,46,54,2,59,11,25,43,29,31,8,59,2,11,19,56,35,57,13,2,35,6,54,17,0,49,5,15,3,60,44,16,6,57,44,58,54,26,23,58,23,26,29,56,40,45,2,21,9,57,40,4,46,17,15,62,21,5,54,22,47,10,24,53,61,43,52,23,10,61,43,26,31,38,2}), + createColumn({4286230172992429668ull,40,21,1096164497041ull,486836301432020572ull,601370,489134489,572,1647797,333708,82422368583289ull,31751841,4296,721,26420894492846ull,255,247,12876009029165923ull,106656820,13398,236546334147978ull,5969577224916561ull,46185410,10555082060194839563ull,443830342103390662ull,6294246,129109,886,472558346614824758ull,4,337609341558356ull,29385104150ull,759076,5293054133ull,3019197118ull,49023884691732699ull,3,3751671909828422707ull,3146297760588474ull,26032891996838ull,2,77690599,106,1093447597522806ull,2621096301034322684ull,360610038,213371041906435251ull,232,75521032470284ull,16701902679050716746ull,26651,486085132525252561ull,236647287356208ull,2007621004085024599ull,10,878950,45650842722325ull,255337486239770279ull,103,999862,52,903,217819909738ull,55662047251ull,36,30465023560ull,88852490364ull,14909735319ull,26,5956433,498857,837436059313328659ull,1371716826717ull,32798405027192516ull,112,16126825,81586030353603970ull,50715,76875338920813ull,36811471868177ull,0,2439873341049ull,131759532317425638ull,22,2683710990390ull,76640,8387068003153235ull,379169582252ull,527,5,1577031,2763,198914727930ull,9914959051251108ull,7,1476603,35048777915ull,4998886136ull,54520161,4412766108116611025ull})); + // clang-format on +} +CATCH + +TEST_F(TestFunctionBitShiftRight, UB) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn({127, -128}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({127, -128}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({32767, -32768}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({INT32_MAX, INT32_MIN}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({INT64_MAX, INT64_MIN}), createColumn({64, 64}), createColumn({0, 0})); + + ASSERT_BITSHIFTRIGHT(createColumn({255}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({255}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({65535}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({UINT32_MAX}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({UINT64_MAX}), createColumn({64}), createColumn({0})); + + /* + std::mt19937 gen(std::random_device{}()); + std::uniform_int_distribution dis1( + std::numeric_limits::min(), + std::numeric_limits::max() + ); + std::uniform_int_distribution dis2( + 64, + std::numeric_limits::max() + ); + size_t count = 100; + std::vector v1(count), v2(count), res(count); + for (size_t i=0; i({17563387625296433369ull,5842891814427459261ull,15074502074821508463ull,386435802999553003ull,5487893274931198395ull,8125923807366590570ull,13340330062727071249ull,14908193031091561411ull,296805448857369387ull,8684453485792353774ull,13117933444495098288ull,3225762988982100714ull,11290506757949810556ull,14617912756126856962ull,9479575714707174581ull,11720728318194739598ull,14410575429605211363ull,12068356718035872518ull,80682389916710599ull,11003236134534292734ull,4412447398096224810ull,5331184707993902906ull,13827083432789678788ull,958142831027309576ull,16716461997317184701ull,17128750834581527743ull,11590434571174666313ull,10204342520615148287ull,11067791415848657283ull,17583875436196878829ull,186304014359496415ull,9381729025189804702ull,11502205568225715300ull,16472133582690439104ull,3743303387826342067ull,12860029445868505658ull,2244056593742923769ull,3275687468466891223ull,1545828456957460699ull,14187252460708728077ull,7551907967738536187ull,9754400233340010491ull,16293183350230169116ull,6298812696728711031ull,5915538565572009956ull,2284684518775825662ull,1130711226902262476ull,17158957721471765323ull,4220824385439711070ull,16559772875254313109ull,15397179690017513678ull,6300413832999049491ull,13787530251307637715ull,10132349060092695582ull,10446586881482901699ull,15759779838283537085ull,14402587207027333363ull,5546051719872960161ull,6545031029710296628ull,17407295406267098658ull,4259019625544816073ull,791895457880289787ull,8549227257401578066ull,15246278171168501125ull,1674668228908076954ull,849762797502000057ull,13302651500925764574ull,12438174880334092333ull,17701249772557033303ull,10742459186038873636ull,15671491258945407856ull,9352557101631889001ull,8914093883925002585ull,17935292744735591949ull,606989231583658922ull,6528503454270721815ull,14980539549624989095ull,13765196438235456668ull,3058323869228644592ull,14346577759191739044ull,1543206286382906519ull,1025562312317433790ull,17052896445025268012ull,18349597294988935754ull,17174604730104962524ull,11924965352621110201ull,502032511104181724ull,13845633389643139332ull,15436039204445155412ull,17809579006694175565ull,15166364145138562881ull,14062748599121933798ull,1777457178576774356ull,4985224560472716170ull,3881603168175384251ull,11555031280550342082ull,1252677486917153396ull,8744807353133366467ull,2048964426549800495ull,11945831330508218140ull}), + createColumn({7570379165150948640ull,2086259313016069849ull,3606689596671293211ull,14039117280692395662ull,13678665403528829741ull,16069000531561010558ull,18229345530821449414ull,433464578739092378ull,6298872104011095934ull,4518228872693063137ull,14988726875963869472ull,9568218424260764817ull,5383191468426384555ull,8698762658876708752ull,9487599666567205013ull,14370091126330876161ull,10702068376663045773ull,8045701071228357739ull,10878469353312437370ull,3183167829827610494ull,5928881618833110378ull,10410530709181481816ull,249988564503361262ull,13482614555530280987ull,5522946068620734806ull,12797173590813112894ull,14133419908717831141ull,10825732602137508628ull,13271177233899692778ull,1157753039017783757ull,3370600557036147696ull,2957689395775524062ull,11963898745206689513ull,4828931188614542720ull,15157289330857160797ull,369467010700905309ull,6278071805692607460ull,17817858137511910604ull,17789013631125929528ull,2861684947245777353ull,2583152408663154190ull,7935135702156687355ull,3033127046167579202ull,14224256960933395097ull,10838403249753694181ull,2154089102842257532ull,7860358918492191001ull,2982010253383852617ull,16385171982396620123ull,12241857497176342828ull,2080931105225959532ull,1046322072991155713ull,6146917059052005252ull,17411786298437646544ull,5497869583209795613ull,11701448129764809247ull,12642962700918363620ull,15936842187305218463ull,7811510447588439153ull,3558405966224377785ull,977960926168429540ull,9505800334935014018ull,12114068456102275321ull,5141880021314950000ull,6719615890604904521ull,1341445859098821585ull,3883912906202435997ull,2107770591867486616ull,2657186337437393032ull,2640917573672927653ull,3746140861437224253ull,15057648507099656234ull,12051189681068107042ull,2259769676757597701ull,2935229535510718769ull,6368233316971463582ull,14384644474340782197ull,2553547617837260603ull,14238122466576902747ull,9555765226032904481ull,15522640015319979866ull,10274396157562093026ull,5996101113505388770ull,16915812546351047056ull,4956089714130804219ull,17126605744801075545ull,12036643325202409080ull,11257234688654558199ull,375338337104024778ull,11152980243617851986ull,12325805905403174063ull,8653948654121626815ull,15348912598299408338ull,6883296938248095081ull,6484642948886870833ull,16936141613107270500ull,17012171815528507292ull,2574129622316042070ull,17178726110735453748ull,16578303277501346489ull}), + createColumn({0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0})); + // clang-format on +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Functions/tests/gtest_duration_pushdown.cpp b/dbms/src/Functions/tests/gtest_duration_pushdown.cpp index 4501a4c9fae..106f3d84642 100644 --- a/dbms/src/Functions/tests/gtest_duration_pushdown.cpp +++ b/dbms/src/Functions/tests/gtest_duration_pushdown.cpp @@ -166,5 +166,85 @@ try ASSERT_COLUMN_EQ(microSecond_out, executeFunction("microSecond", input4)); } CATCH + +TEST_F(DurationPushDown, timeToSecPushDownTest) +try +{ + ColumnWithTypeAndName input( + createColumn>({(838 * 3600 + 59 * 60 + 59) * 1000000000L + 999999000L, + -(838 * 3600 + 59 * 60 + 59) * 1000000000L - 123456000L, + 0, + (1 * 3600 + 2 * 60 + 3) * 1000000000L + 4000L}) + .column, + makeNullable(std::make_shared(6)), + "input"); + auto second_output = createColumn>({3020399, -3020399, 0, 3723}); + ASSERT_COLUMN_EQ(second_output, executeFunction("tidbTimeToSec", input)); + + // Test Overflow + ColumnWithTypeAndName input2( + createColumn>({(838 * 3600 + 59 * 60 + 59) * 1000000000L + 999999000L + 1000L}).column, + makeNullable(std::make_shared(6)), + "result"); + try + { + auto result = executeFunction("tidbTimeToSec", input2); + FAIL() << "Expected overflow"; + } + catch (DB::Exception & e) + { + ASSERT_EQ(e.message(), std::string("nanos must >= -3020399999999000 and <= 3020399999999000")); + } + catch (...) + { + FAIL() << "Expected overflow"; + }; + + ColumnWithTypeAndName input3( + createColumn>({-(838 * 3600 + 59 * 60 + 59) * 1000000000L - 999999000L - 1000L}).column, + makeNullable(std::make_shared(6)), + "result"); + try + { + auto result = executeFunction("tidbTimeToSec", input3); + FAIL() << "Expected overflow"; + } + catch (DB::Exception & e) + { + ASSERT_EQ(e.message(), std::string("nanos must >= -3020399999999000 and <= 3020399999999000")); + } + catch (...) + { + FAIL() << "Expected overflow"; + }; + + // Random Test + constexpr int rowNum = 1000; + auto dur_column = ColumnVector::create(); + auto & dur_data = dur_column->getData(); + auto second_column = ColumnVector::create(); + auto & second_data = second_column->getData(); + dur_data.resize(rowNum); + second_data.resize(rowNum); + + std::random_device rd; + std::default_random_engine gen = std::default_random_engine(rd()); + std::uniform_int_distribution sign_dis(0, 1), hour_dis(0, 838), minute_dis(0, 59), second_dis(0, 59), microSecond_dis(0, 999999); + for (int i = 0; i < rowNum; ++i) + { + auto sign = (sign_dis(gen) == 0) ? 1 : -1; + auto hour = hour_dis(gen); + auto minute = minute_dis(gen); + auto second = second_dis(gen); + auto microSecond = microSecond_dis(gen); + dur_data[i] = sign * ((hour * 3600 + minute * 60 + second) * 1000000000L + microSecond * 1000L); + second_data[i] = sign * (hour * 3600 + minute * 60 + second); + } + + ColumnWithTypeAndName input4(std::move(dur_column), std::make_shared(6), "duration"); + ColumnWithTypeAndName second_out(std::move(second_column), std::make_shared(), "time_to_sec"); + ASSERT_COLUMN_EQ(second_out, executeFunction("tidbTimeToSec", input4)); +} +CATCH } // namespace tests } // namespace DB \ No newline at end of file diff --git a/dbms/src/Functions/tests/gtest_strings_reverse.cpp b/dbms/src/Functions/tests/gtest_strings_reverse.cpp new file mode 100644 index 00000000000..304a403db83 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_strings_reverse.cpp @@ -0,0 +1,120 @@ +// 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 + +#pragma GCC diagnostic pop + +namespace DB::tests +{ +class StringReverse : public DB::tests::FunctionTest +{ +protected: + static ColumnWithTypeAndName toVec(const std::vector & v) + { + return createColumn(v); + } + + static ColumnWithTypeAndName toNullableVec(const std::vector> & v) + { + return createColumn>(v); + } + + static ColumnWithTypeAndName toConst(const String & s) + { + return createConstColumn(1, s); + } +}; +// test reverse +TEST_F(StringReverse, stringReverseTest) +try +{ + std::vector candidate_strings = {"one week's time test", "abcdef", "abcabc", "moc.pacgnip"}; + std::vector reversed_strings = {"tset emit s'keew eno", "fedcba", "cbacba", "pingcap.com"}; + + // test vector + ASSERT_COLUMN_EQ( + toVec(reversed_strings), + executeFunction( + "reverse", + toVec(candidate_strings))); + + // test nullable + ASSERT_COLUMN_EQ( + toNullableVec({"", " ", {}, "pacgnip"}), + executeFunction( + "reverse", + toNullableVec({"", " ", {}, "pingcap"}))); + + // test const + ASSERT_COLUMN_EQ( + toConst("pacgnip"), + executeFunction( + "reverse", + toConst("pingcap"))); + + // test null + ASSERT_COLUMN_EQ( + toConst({}), + executeFunction( + "reverse", + toConst({}))); +} +CATCH + +// test reverseUTF8 +TEST_F(StringReverse, stringReverseUTF8Test) +try +{ + std::vector candidate_strings = {"one week's time test", "abc测试def", "abcテストabc", "ѐёђѓєѕіїјљњћќѝўџ", "+ѐ-ё*ђ/ѓ!є@ѕ#і$@ї%ј……љ&њ(ћ)ќ¥ѝ#ў@џ!^", "αβγδεζηθικλμνξοπρστυφχψωσ", "▲α▼βγ➨δε☎ζη✂θι€κλ♫μν✓ξο✚πρ℉στ♥υφ♖χψ♘ω★σ✕", "թփձջրչճժծքոեռտըւիօպասդֆգհյկլխզղցվբնմշ"}; + std::vector reversed_strings = {"tset emit s'keew eno", "fed试测cba", "cbaトステcba", "џўѝќћњљјїіѕєѓђёѐ", "^!џ@ў#ѝ¥ќ)ћ(њ&љ……ј%ї@$і#ѕ@є!ѓ/ђ*ё-ѐ+", "σωψχφυτσρποξνμλκιθηζεδγβα", "✕σ★ω♘ψχ♖φυ♥τσ℉ρπ✚οξ✓νμ♫λκ€ιθ✂ηζ☎εδ➨γβ▼α▲", "շմնբվցղզխլկյհգֆդսապօիւըտռեոքծժճչրջձփթ"}; + + // test vector + ASSERT_COLUMN_EQ( + toVec(reversed_strings), + executeFunction( + "reverseUTF8", + toVec(candidate_strings))); + + // test nullable + ASSERT_COLUMN_EQ( + toNullableVec({"", " ", {}, "pacgnip"}), + executeFunction( + "reverseUTF8", + toNullableVec({"", " ", {}, "pingcap"}))); + + // test const + ASSERT_COLUMN_EQ( + toConst("pacgnip"), + executeFunction( + "reverseUTF8", + toConst("pingcap"))); + + // test null + ASSERT_COLUMN_EQ( + toConst({}), + executeFunction( + "reverseUTF8", + toConst({}))); +} +CATCH + +} // namespace DB::tests \ No newline at end of file diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 3358ae2a60e..a398aa9c74d 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -178,6 +178,12 @@ namespace } } // namespace +namespace CurrentMetrics +{ +extern const Metric LogicalCPUCores; +extern const Metric MemoryCapacity; +} // namespace CurrentMetrics + namespace DB { namespace ErrorCodes @@ -1427,6 +1433,8 @@ int Server::main(const std::vector & /*args*/) { // on ARM processors it can show only enabled at current moment cores + CurrentMetrics::set(CurrentMetrics::LogicalCPUCores, server_info.cpu_info.logical_cores); + CurrentMetrics::set(CurrentMetrics::MemoryCapacity, server_info.memory_info.capacity); LOG_FMT_INFO( log, "Available RAM = {}; physical cores = {}; logical cores = {}.", diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp index 1c68ba3bb2a..dc43ef3713b 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -75,12 +75,26 @@ CATCH TEST_F(SegmentOperationTest, TestSegmentRandom) try { + srand(time(nullptr)); SegmentTestOptions options; options.is_common_handle = true; reloadWithOptions(options); randomSegmentTest(100); } CATCH + +// run in CI weekly +TEST_F(SegmentOperationTest, DISABLED_TestSegmentRandomForCI) +try +{ + srand(time(nullptr)); + SegmentTestOptions options; + options.is_common_handle = true; + reloadWithOptions(options); + randomSegmentTest(10000); +} +CATCH + } // namespace tests } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/Transaction/Collator.cpp b/dbms/src/Storages/Transaction/Collator.cpp index a9b4d0784be..1b0221a6829 100644 --- a/dbms/src/Storages/Transaction/Collator.cpp +++ b/dbms/src/Storages/Transaction/Collator.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include @@ -29,17 +30,10 @@ TiDBCollators dummy_collators; std::vector dummy_sort_key_contaners; std::string dummy_sort_key_contaner; -std::string_view rtrim(const char * s, size_t length) +ALWAYS_INLINE std::string_view rtrim(const char * s, size_t length) { auto v = std::string_view(s, length); - size_t end = v.find_last_not_of(' '); - return end == std::string_view::npos ? "" : v.substr(0, end + 1); -} - -template -int signum(T val) -{ - return (0 < val) - (val < 0); + return DB::RightTrim(v); } using Rune = int32_t; @@ -183,26 +177,26 @@ class Pattern : public ITiDBCollator::IPattern }; template -class BinCollator : public ITiDBCollator +class BinCollator final : public ITiDBCollator { public: explicit BinCollator(int32_t id) : ITiDBCollator(id) {} + int compare(const char * s1, size_t length1, const char * s2, size_t length2) const override { if constexpr (padding) - return signum(rtrim(s1, length1).compare(rtrim(s2, length2))); + return DB::RtrimStrCompare({s1, length1}, {s2, length2}); else - return signum(std::string_view(s1, length1).compare(std::string_view(s2, length2))); + return DB::RawStrCompare({s1, length1}, {s2, length2}); } StringRef sortKey(const char * s, size_t length, std::string &) const override { if constexpr (padding) { - auto v = rtrim(s, length); - return StringRef(v.data(), v.length()); + return StringRef(rtrim(s, length)); } else { @@ -249,7 +243,7 @@ using WeightType = uint16_t; extern const std::array weight_lut; } // namespace GeneralCI -class GeneralCICollator : public ITiDBCollator +class GeneralCICollator final : public ITiDBCollator { public: explicit GeneralCICollator(int32_t id) @@ -270,7 +264,7 @@ class GeneralCICollator : public ITiDBCollator auto sk2 = weight(c2); auto cmp = sk1 - sk2; if (cmp != 0) - return signum(cmp); + return DB::signum(cmp); } return (offset1 < v1.length()) - (offset2 < v2.length()); @@ -365,7 +359,7 @@ const std::array weight_lut_long = { } // namespace UnicodeCI -class UnicodeCICollator : public ITiDBCollator +class UnicodeCICollator final : public ITiDBCollator { public: explicit UnicodeCICollator(int32_t id) @@ -420,7 +414,7 @@ class UnicodeCICollator : public ITiDBCollator } else { - return signum(static_cast(s1_first & 0xFFFF) - static_cast(s2_first & 0xFFFF)); + return DB::signum(static_cast(s1_first & 0xFFFF) - static_cast(s2_first & 0xFFFF)); } } } @@ -593,6 +587,8 @@ class UnicodeCICollator : public ITiDBCollator friend class Pattern; }; +using UTF8MB4_BIN_TYPE = BinCollator; + TiDBCollatorPtr ITiDBCollator::getCollator(int32_t id) { switch (id) @@ -607,10 +603,10 @@ TiDBCollatorPtr ITiDBCollator::getCollator(int32_t id) static const auto latin1_collator = BinCollator(LATIN1_BIN); return &latin1_collator; case ITiDBCollator::UTF8MB4_BIN: - static const auto utf8mb4_collator = BinCollator(UTF8MB4_BIN); + static const auto utf8mb4_collator = UTF8MB4_BIN_TYPE(UTF8MB4_BIN); return &utf8mb4_collator; case ITiDBCollator::UTF8_BIN: - static const auto utf8_collator = BinCollator(UTF8_BIN); + static const auto utf8_collator = UTF8MB4_BIN_TYPE(UTF8_BIN); return &utf8_collator; case ITiDBCollator::UTF8_GENERAL_CI: static const auto utf8_general_ci_collator = GeneralCICollator(UTF8_GENERAL_CI); diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index f9d6d01955e..fb31e4476bb 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -327,6 +327,64 @@ void KVStore::persistRegion(const Region & region, const RegionTaskLock & region LOG_FMT_DEBUG(log, "Persist {} done", region.toString(false)); } +bool KVStore::needFlushRegionData(UInt64 region_id, TMTContext & tmt) +{ + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + const RegionPtr curr_region_ptr = getRegion(region_id); + return canFlushRegionDataImpl(curr_region_ptr, false, false, tmt, region_task_lock); +} + +bool KVStore::tryFlushRegionData(UInt64 region_id, bool try_until_succeed, TMTContext & tmt) +{ + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + const RegionPtr curr_region_ptr = getRegion(region_id); + return canFlushRegionDataImpl(curr_region_ptr, true, try_until_succeed, tmt, region_task_lock); +} + +bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock) +{ + if (curr_region_ptr == nullptr) + { + throw Exception(fmt::format("region not found when trying flush", ErrorCodes::LOGICAL_ERROR)); + } + auto & curr_region = *curr_region_ptr; + + auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); + + LOG_FMT_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); + + bool can_flush = false; + if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) + || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) + { + // if rows or bytes more than threshold, flush cache and persist mem data. + can_flush = true; + } + else + { + // if there is little data in mem, wait until time interval reached threshold. + // use random period so that lots of regions will not be persisted at same time. + auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT + can_flush = !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); + } + if (can_flush && flush_if_possible) + { + LOG_FMT_DEBUG(log, "{} flush region due to can_flush_data", curr_region.toString(false)); + if (tryFlushRegionCacheInStorage(tmt, curr_region, log, try_until_succeed)) + { + persistRegion(curr_region, region_task_lock, "compact raft log"); + curr_region.markCompactLog(); + curr_region.cleanApproxMemCacheInfo(); + return true; + } + else + { + return false; + } + } + return can_flush; +} + EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( raft_cmdpb::AdminCmdType cmd_type, UInt64 curr_region_id, @@ -360,39 +418,13 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( } else { - auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); - - LOG_FMT_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); - - if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) - || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) - { - // if rows or bytes more than threshold, try to flush cache and persist mem data. - return true; - } - else - { - // if there is little data in mem, wait until time interval reached threshold. - // use random period so that lots of regions will not be persisted at same time. - auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT - return !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); - } + return canFlushRegionDataImpl(curr_region_ptr, true, /* try_until_succeed */ false, tmt, region_task_lock); } }; if (check_sync_log()) { - if (tryFlushRegionCacheInStorage(tmt, curr_region, log, /* try_until_succeed */ false)) - { - persistRegion(curr_region, region_task_lock, "compact raft log"); - curr_region.markCompactLog(); - curr_region.cleanApproxMemCacheInfo(); - return EngineStoreApplyRes::Persist; - } - else - { - return EngineStoreApplyRes::None; - } + return EngineStoreApplyRes::Persist; } return EngineStoreApplyRes::None; } diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 66e2fe32b75..b58083557a1 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -108,6 +108,9 @@ class KVStore final : private boost::noncopyable TMTContext & tmt); EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); + bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); + bool tryFlushRegionData(UInt64 region_id, bool try_until_succeed, TMTContext & tmt); + void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); std::vector /* */ preHandleSnapshotToFiles( @@ -219,6 +222,11 @@ class KVStore final : private boost::noncopyable UInt64 term, TMTContext & tmt); + /// Notice that if flush_if_possible is set to false, we only check if a flush is allowed by rowsize/size/interval. + /// It will not check if a flush will eventually succeed. + /// In other words, `canFlushRegionDataImpl(flush_if_possible=true)` can return false. + bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock); + void persistRegion(const Region & region, const RegionTaskLock & region_task_lock, const char * caller); void releaseReadIndexWorkers(); void handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTaskLock &); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 8a40ca9b15e..d4ba50d5714 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -128,6 +128,34 @@ EngineStoreApplyRes HandleAdminRaftCmd( } } +uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id) +{ + try + { + auto & kvstore = server->tmt->getKVStore(); + return kvstore->needFlushRegionData(region_id, *server->tmt); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + exit(-1); + } +} + +uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t until_succeed) +{ + try + { + auto & kvstore = server->tmt->getKVStore(); + return kvstore->tryFlushRegionData(region_id, until_succeed, *server->tmt); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + exit(-1); + } +} + static_assert(sizeof(RaftStoreProxyFFIHelper) == sizeof(TiFlashRaftProxyHelper)); static_assert(alignof(RaftStoreProxyFFIHelper) == alignof(TiFlashRaftProxyHelper)); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index e1c01599275..aafe4b375eb 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -125,6 +125,8 @@ EngineStoreApplyRes HandleAdminRaftCmd( EngineStoreApplyRes HandleWriteRaftCmd(const EngineStoreServerWrap * server, WriteCmdsView cmds, RaftCmdHeader header); +uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id); +uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t until_succeed); void AtomicUpdateProxy(EngineStoreServerWrap * server, RaftStoreProxyFFIHelper * proxy); void HandleDestroy(EngineStoreServerWrap * server, uint64_t region_id); EngineStoreApplyRes HandleIngestSST(EngineStoreServerWrap * server, SSTViewVec snaps, RaftCmdHeader header); @@ -158,6 +160,8 @@ inline EngineStoreServerHelper GetEngineStoreServerHelper( .fn_gen_cpp_string = GenCppRawString, .fn_handle_write_raft_cmd = HandleWriteRaftCmd, .fn_handle_admin_raft_cmd = HandleAdminRaftCmd, + .fn_need_flush_data = NeedFlushData, + .fn_try_flush_data = TryFlushData, .fn_atomic_update_proxy = AtomicUpdateProxy, .fn_handle_destroy = HandleDestroy, .fn_handle_ingest_sst = HandleIngestSST, diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index dc7f1f3e348..6d07c47f235 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include @@ -772,6 +773,37 @@ 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 ////// /////////////////////// @@ -840,6 +872,8 @@ try json->set("tiflash_replica", replica_info.getJSONObject()); + json->set("tiflash_mode", std::string(TiFlashModeToString(tiflash_mode))); + json->stringify(buf); return buf.str(); @@ -926,6 +960,14 @@ 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( diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index 4c28a614857..a9d46b60c13 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -333,6 +333,12 @@ struct IndexInfo bool is_global; }; +enum class TiFlashMode +{ + Normal, + Fast, +}; + struct TableInfo { TableInfo() = default; @@ -382,6 +388,8 @@ 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; @@ -413,4 +421,7 @@ String genJsonNull(); tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci); ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); +String TiFlashModeToString(TiFlashMode tiflash_mode); +TiFlashMode parseTiFlashMode(String mode_str); + } // namespace TiDB diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 36a91522bb6..77aab06f6cf 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -1179,6 +1179,12 @@ void RegionKVStoreTest::testKVStore() ASSERT_EQ(e.message(), "unsupported admin command type InvalidAdmin"); } } + { + // There shall be data to flush. + ASSERT_EQ(kvs.needFlushRegionData(19, ctx.getTMTContext()), true); + // Force flush until succeed only for testing. + ASSERT_EQ(kvs.tryFlushRegionData(19, true, ctx.getTMTContext()), true); + } } void test_mergeresult() diff --git a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp index 516a173b151..871153cb0e9 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp @@ -42,7 +42,7 @@ struct ParseCase std::function check; }; -TEST(TiDBTableInfo_test, ParseFromJSON) +TEST(TiDBTableInfoTest, ParseFromJSON) try { auto cases = { @@ -136,54 +136,54 @@ struct StmtCase } }; -TEST(TiDBTableInfo_test, GenCreateTableStatement) +TEST(TiDBTableInfoTest, GenCreateTableStatement) try { auto cases = // {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})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", // + 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", // }, 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})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", // + 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", // }, 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})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", // + 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", // }, 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})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", // + 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", // }, 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})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", // + 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", // }, 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})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", // + 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", // }, 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})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", // + 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", // }}; - for (auto & c : cases) + for (const auto & c : cases) { c.verifyTableInfo(); } diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 9a6e92dd9c1..49ae9787ea4 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -35,7 +35,7 @@ ASTPtr buildLiteral(const Field & field) return std::make_shared(field); } -ASTPtr buildOrderByItemList(MockOrderByItems order_by_items) +ASTPtr buildOrderByItemVec(MockOrderByItemVec order_by_items) { std::vector vec(order_by_items.size()); size_t i = 0; @@ -92,7 +92,7 @@ std::shared_ptr DAGRequestBuilder::build(MockDAGRequestContext return dag_request_ptr; } -DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String & table, const MockColumnInfos & columns) +DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String & table, const MockColumnInfoVec & columns) { assert(!columns.empty()); TableInfo table_info; @@ -114,27 +114,17 @@ DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String return *this; } -DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, const MockColumnInfos & columns) +DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, const MockColumnInfoVec & columns) { return mockTable(name.first, name.second, columns); } -DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, const MockColumnInfoList & columns) -{ - return mockTable(name.first, name.second, columns); -} - -DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfos & columns) -{ - return buildExchangeReceiver(columns); -} - -DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoList & columns) +DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoVec & columns) { return buildExchangeReceiver(columns); } -DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfos & columns) +DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfoVec & columns) { DAGSchema schema; for (const auto & column : columns) @@ -180,33 +170,23 @@ DAGRequestBuilder & DAGRequestBuilder::topN(ASTPtr order_exprs, ASTPtr limit_exp DAGRequestBuilder & DAGRequestBuilder::topN(const String & col_name, bool desc, int limit) { assert(root); - root = compileTopN(root, getExecutorIndex(), buildOrderByItemList({{col_name, desc}}), buildLiteral(Field(static_cast(limit)))); + root = compileTopN(root, getExecutorIndex(), buildOrderByItemVec({{col_name, desc}}), buildLiteral(Field(static_cast(limit)))); return *this; } -DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItems order_by_items, int limit) +DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItemVec order_by_items, int limit) { return topN(order_by_items, buildLiteral(Field(static_cast(limit)))); } -DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItems order_by_items, ASTPtr limit_expr) +DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItemVec order_by_items, ASTPtr limit_expr) { assert(root); - root = compileTopN(root, getExecutorIndex(), buildOrderByItemList(order_by_items), limit_expr); + root = compileTopN(root, getExecutorIndex(), buildOrderByItemVec(order_by_items), limit_expr); return *this; } -DAGRequestBuilder & DAGRequestBuilder::project(const String & col_name) -{ - assert(root); - auto exp_list = std::make_shared(); - exp_list->children.push_back(buildColumn(col_name)); - - root = compileProject(root, getExecutorIndex(), exp_list); - return *this; -} - -DAGRequestBuilder & DAGRequestBuilder::project(MockAsts exprs) +DAGRequestBuilder & DAGRequestBuilder::project(MockAstVec exprs) { assert(root); auto exp_list = std::make_shared(); @@ -218,12 +198,7 @@ DAGRequestBuilder & DAGRequestBuilder::project(MockAsts exprs) return *this; } -DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNames col_names) -{ - return project(MockColumnNamesVec(col_names)); -} - -DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNamesVec col_names) +DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNameVec col_names) { assert(root); auto exp_list = std::make_shared(); @@ -242,12 +217,12 @@ DAGRequestBuilder & DAGRequestBuilder::exchangeSender(tipb::ExchangeType exchang return *this; } -DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAsts exprs) +DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAstVec exprs) { return join(right, exprs, ASTTableJoin::Kind::Inner); } -DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAsts exprs, ASTTableJoin::Kind kind) +DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAstVec exprs, ASTTableJoin::Kind kind) { assert(root); assert(right.root); @@ -273,7 +248,7 @@ DAGRequestBuilder & DAGRequestBuilder::aggregation(ASTPtr agg_func, ASTPtr group return buildAggregation(agg_funcs, group_by_exprs); } -DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAsts agg_funcs, MockAsts group_by_exprs) +DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs) { auto agg_func_list = std::make_shared(); auto group_by_expr_list = std::make_shared(); @@ -296,80 +271,58 @@ DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByIte assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList({partition_by}), buildOrderByItemList({order_by}), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec({partition_by}), buildOrderByItemVec({order_by}), frame); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) { assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList(partition_by_list), buildOrderByItemList(order_by_list), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(MockAsts window_funcs, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) { assert(root); auto window_func_list = std::make_shared(); for (const auto & func : window_funcs) window_func_list->children.push_back(func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList(partition_by_list), buildOrderByItemList(order_by_list), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); return *this; } DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemList({order_by}), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec({order_by}), is_partial_sort); return *this; } -DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItems order_by_list, bool is_partial_sort) +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItemVec order_by_vec, bool is_partial_sort) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemList(order_by_list), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec(order_by_vec), is_partial_sort); return *this; } -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos) -{ - std::vector v_column_info(columnInfos.size()); - size_t i = 0; - for (const auto & info : columnInfos) - { - v_column_info[i++] = std::move(info); - } - mock_tables[name.first + "." + name.second] = v_column_info; -} - -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos) +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos) { mock_tables[db + "." + table] = columnInfos; } -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos) +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos) { mock_tables[name.first + "." + name.second] = columnInfos; } -void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfos & columnInfos) +void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfoVec & columnInfos) { exchange_schemas[name] = columnInfos; } -void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfoList & columnInfos) -{ - std::vector v_column_info(columnInfos.size()); - size_t i = 0; - for (const auto & info : columnInfos) - { - v_column_info[i++] = std::move(info); - } - exchange_schemas[name] = v_column_info; -} - void MockDAGRequestContext::addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns) { mock_table_columns[db + "." + table] = columns; @@ -385,37 +338,19 @@ void MockDAGRequestContext::addExchangeReceiverColumnData(const String & name, C mock_exchange_columns[name] = columns; } -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns) { addMockTable(db, table, columnInfos); addMockTableColumnData(db, table, columns); } -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(db, table, columnInfos); - addMockTableColumnData(db, table, columns); -} - -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(name, columnInfos); - addMockTableColumnData(name, columns); -} - -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns) { addMockTable(name, columnInfos); addMockTableColumnData(name, columns); } -void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns) -{ - addExchangeRelationSchema(name, columnInfos); - addExchangeReceiverColumnData(name, columns); -} - -void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns) +void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfoVec columnInfos, ColumnsWithTypeAndName columns) { addExchangeRelationSchema(name, columnInfos); addExchangeReceiverColumnData(name, columns); diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index bad92c4226d..971061e93e7 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -23,16 +23,14 @@ namespace DB::tests { using MockColumnInfo = std::pair; -using MockColumnInfos = std::vector; -using MockColumnInfoList = std::initializer_list; +using MockColumnInfoVec = std::vector; using MockTableName = std::pair; using MockOrderByItem = std::pair; -using MockOrderByItems = std::initializer_list; +using MockOrderByItemVec = std::vector; using MockPartitionByItem = std::pair; -using MockPartitionByItems = std::initializer_list; -using MockColumnNames = std::initializer_list; -using MockColumnNamesVec = std::vector; -using MockAsts = std::initializer_list; +using MockPartitionByItemVec = std::vector; +using MockColumnNameVec = std::vector; +using MockAstVec = std::vector; using MockWindowFrame = mock::MockWindowFrame; class MockDAGRequestContext; @@ -65,12 +63,10 @@ class DAGRequestBuilder std::shared_ptr build(MockDAGRequestContext & mock_context); - DAGRequestBuilder & mockTable(const String & db, const String & table, const MockColumnInfos & columns); - DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfos & columns); - DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfoList & columns); + DAGRequestBuilder & mockTable(const String & db, const String & table, const MockColumnInfoVec & columns); + DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfoVec & columns); - DAGRequestBuilder & exchangeReceiver(const MockColumnInfos & columns); - DAGRequestBuilder & exchangeReceiver(const MockColumnInfoList & columns); + DAGRequestBuilder & exchangeReceiver(const MockColumnInfoVec & columns); DAGRequestBuilder & filter(ASTPtr filter_expr); @@ -79,36 +75,34 @@ class DAGRequestBuilder DAGRequestBuilder & topN(ASTPtr order_exprs, ASTPtr limit_expr); DAGRequestBuilder & topN(const String & col_name, bool desc, int limit); - DAGRequestBuilder & topN(MockOrderByItems order_by_items, int limit); - DAGRequestBuilder & topN(MockOrderByItems order_by_items, ASTPtr limit_expr); + DAGRequestBuilder & topN(MockOrderByItemVec order_by_items, int limit); + DAGRequestBuilder & topN(MockOrderByItemVec order_by_items, ASTPtr limit_expr); - DAGRequestBuilder & project(const String & col_name); - DAGRequestBuilder & project(MockAsts expr); - DAGRequestBuilder & project(MockColumnNames col_names); - DAGRequestBuilder & project(MockColumnNamesVec col_names); + DAGRequestBuilder & project(MockAstVec exprs); + DAGRequestBuilder & project(MockColumnNameVec col_names); DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type); // Currentlt only support inner join, left join and right join. // TODO support more types of join. - DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAsts exprs); - DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAsts exprs, ASTTableJoin::Kind kind); + DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs); + DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs, ASTTableJoin::Kind kind); // aggregation DAGRequestBuilder & aggregation(ASTPtr agg_func, ASTPtr group_by_expr); - DAGRequestBuilder & aggregation(MockAsts agg_funcs, MockAsts group_by_exprs); + DAGRequestBuilder & aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs); // window DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame); - DAGRequestBuilder & window(MockAsts window_funcs, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame); - DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame); + DAGRequestBuilder & window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort); - DAGRequestBuilder & sort(MockOrderByItems order_by_list, bool is_partial_sort); + DAGRequestBuilder & sort(MockOrderByItemVec order_by_vec, bool is_partial_sort); private: void initDAGRequest(tipb::DAGRequest & dag_request); DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); - DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfos & columns); + DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns); ExecutorPtr root; DAGProperties properties; @@ -132,20 +126,15 @@ class MockDAGRequestContext return DAGRequestBuilder(index); } - void addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos); - void addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos); - void addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos); - void addExchangeRelationSchema(String name, const MockColumnInfos & columnInfos); - void addExchangeRelationSchema(String name, const MockColumnInfoList & columnInfos); + void addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos); + void addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos); + void addExchangeRelationSchema(String name, const MockColumnInfoVec & columnInfos); void addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns); - void addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns); - void addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns); - void addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns); - void addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns); void addMockTableColumnData(const MockTableName & name, ColumnsWithTypeAndName columns); void addExchangeReceiverColumnData(const String & name, ColumnsWithTypeAndName columns); - void addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns); - void addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns); + void addExchangeReceiver(const String & name, MockColumnInfoVec columnInfos, ColumnsWithTypeAndName columns); std::unordered_map & executorIdColumnsMap() { return executor_id_columns_map; } @@ -154,8 +143,8 @@ class MockDAGRequestContext private: size_t index; - std::unordered_map mock_tables; - std::unordered_map exchange_schemas; + std::unordered_map mock_tables; + std::unordered_map exchange_schemas; std::unordered_map mock_table_columns; std::unordered_map mock_exchange_columns; std::unordered_map executor_id_columns_map; /// @@ -170,8 +159,8 @@ class MockDAGRequestContext ASTPtr buildColumn(const String & column_name); ASTPtr buildLiteral(const Field & field); -ASTPtr buildFunction(MockAsts exprs, const String & name); -ASTPtr buildOrderByItemList(MockOrderByItems order_by_items); +ASTPtr buildFunction(MockAstVec exprs, const String & name); +ASTPtr buildOrderByItemVec(MockOrderByItemVec order_by_items); MockWindowFrame buildDefaultRowsFrame(); diff --git a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp index 8bed0f2fc6c..72f0bb505d1 100644 --- a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp +++ b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp @@ -76,7 +76,7 @@ TEST_F(MockDAGRequestTest, Projection) try { auto request = context.scan("test_db", "test_table") - .project("s1") + .project({"s1"}) .build(context); { String expected = "project_1 | {<0, String>}\n" diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index ae78923fc61..6e4ad10e344 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -320,7 +321,7 @@ inline SchemaChanges detectSchemaChanges( } template -void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_info, TableInfoPtr table_info, ManageableStoragePtr storage) +void SchemaBuilder::applyAlterPhysicalTable(const DBInfoPtr & db_info, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { LOG_FMT_INFO(log, "Altering table {}", name_mapper.debugCanonicalName(*db_info, *table_info)); @@ -394,7 +395,7 @@ void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_inf } template -void SchemaBuilder::applyAlterTable(DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyAlterTable(const DBInfoPtr & db_info, TableID table_id) { auto table_info = getter.getTableInfo(db_info->id, table_id); if (table_info == nullptr) @@ -413,7 +414,7 @@ void SchemaBuilder::applyAlterTable(DBInfoPtr db_info, Table } template -void SchemaBuilder::applyAlterLogicalTable(DBInfoPtr db_info, TableInfoPtr table_info, ManageableStoragePtr storage) +void SchemaBuilder::applyAlterLogicalTable(const DBInfoPtr & db_info, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { // Alter logical table first. applyAlterPhysicalTable(db_info, table_info, storage); @@ -542,6 +543,11 @@ 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) @@ -569,7 +575,7 @@ void SchemaBuilder::applyDiff(const SchemaDiff & diff) } template -void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyPartitionDiff(const TiDB::DBInfoPtr & db_info, TableID table_id) { auto table_info = getter.getTableInfo(db_info->id, table_id); if (table_info == nullptr) @@ -593,7 +599,7 @@ void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_in } template -void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_info, TableInfoPtr table_info, ManageableStoragePtr storage) +void SchemaBuilder::applyPartitionDiff(const TiDB::DBInfoPtr & db_info, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { const auto & orig_table_info = storage->getTableInfo(); if (!orig_table_info.isLogicalPartitionTable()) @@ -659,7 +665,7 @@ void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_in } template -void SchemaBuilder::applyRenameTable(DBInfoPtr new_db_info, TableID table_id) +void SchemaBuilder::applyRenameTable(const DBInfoPtr & new_db_info, TableID table_id) { auto new_table_info = getter.getTableInfo(new_db_info->id, table_id); if (new_table_info == nullptr) @@ -679,9 +685,9 @@ void SchemaBuilder::applyRenameTable(DBInfoPtr new_db_info, template void SchemaBuilder::applyRenameLogicalTable( - DBInfoPtr new_db_info, - TableInfoPtr new_table_info, - ManageableStoragePtr storage) + const DBInfoPtr & new_db_info, + const TableInfoPtr & new_table_info, + const ManageableStoragePtr & storage) { applyRenamePhysicalTable(new_db_info, *new_table_info, storage); @@ -703,9 +709,9 @@ void SchemaBuilder::applyRenameLogicalTable( template void SchemaBuilder::applyRenamePhysicalTable( - DBInfoPtr new_db_info, - TableInfo & new_table_info, - ManageableStoragePtr storage) + const DBInfoPtr & new_db_info, + const TableInfo & new_table_info, + const ManageableStoragePtr & storage) { const auto old_mapped_db_name = storage->getDatabaseName(); const auto new_mapped_db_name = name_mapper.mapDatabaseName(*new_db_info); @@ -908,7 +914,7 @@ String createDatabaseStmt(Context & context, const DBInfo & db_info, const Schem } template -void SchemaBuilder::applyCreateSchema(TiDB::DBInfoPtr db_info) +void SchemaBuilder::applyCreateSchema(const TiDB::DBInfoPtr & db_info) { GET_METRIC(tiflash_schema_internal_ddl_count, type_create_db).Increment(); LOG_FMT_INFO(log, "Creating database {}", name_mapper.debugDatabaseName(*db_info)); @@ -1047,7 +1053,7 @@ String createTableStmt( } template -void SchemaBuilder::applyCreatePhysicalTable(DBInfoPtr db_info, TableInfoPtr table_info) +void SchemaBuilder::applyCreatePhysicalTable(const DBInfoPtr & db_info, const TableInfoPtr & table_info) { GET_METRIC(tiflash_schema_internal_ddl_count, type_create_table).Increment(); LOG_FMT_INFO(log, "Creating table {}", name_mapper.debugCanonicalName(*db_info, *table_info)); @@ -1109,7 +1115,7 @@ void SchemaBuilder::applyCreatePhysicalTable(DBInfoPtr db_in } template -void SchemaBuilder::applyCreateTable(TiDB::DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyCreateTable(const TiDB::DBInfoPtr & db_info, TableID table_id) { auto table_info = getter.getTableInfo(db_info->id, table_id); if (table_info == nullptr) @@ -1123,7 +1129,7 @@ void SchemaBuilder::applyCreateTable(TiDB::DBInfoPtr db_info } template -void SchemaBuilder::applyCreateLogicalTable(TiDB::DBInfoPtr db_info, TableInfoPtr table_info) +void SchemaBuilder::applyCreateLogicalTable(const TiDB::DBInfoPtr & db_info, const TableInfoPtr & table_info) { if (table_info->isLogicalPartitionTable()) { @@ -1169,7 +1175,7 @@ void SchemaBuilder::applyDropPhysicalTable(const String & db } template -void SchemaBuilder::applyDropTable(DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyDropTable(const DBInfoPtr & db_info, TableID table_id) { auto & tmt_context = context.getTMTContext(); auto * storage = tmt_context.getStorages().get(table_id).get(); @@ -1193,13 +1199,14 @@ void SchemaBuilder::applyDropTable(DBInfoPtr db_info, TableI } template -void SchemaBuilder::applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applySetTiFlashReplica(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)) @@ -1208,18 +1215,37 @@ void SchemaBuilder::applySetTiFlashReplica(TiDB::DBInfoPtr d Errors::DDL::MissingTable); } - auto managed_storage = std::dynamic_pointer_cast(storage); - if (unlikely(!managed_storage)) - throw Exception(fmt::format("{} is not a ManageableStorage", name_mapper.debugCanonicalName(*db_info, *latest_table_info))); + applySetTiFlashReplicaOnLogicalTable(db_info, latest_table_info, storage); +} - applySetTiFlashReplica(db_info, latest_table_info, managed_storage); +template +void SchemaBuilder::applySetTiFlashReplicaOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage) +{ + applySetTiFlashReplicaOnPhysicalTable(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); + } + applySetTiFlashReplicaOnPhysicalTable(db_info, new_part_table_info, part_storage); + } + } } template -void SchemaBuilder::applySetTiFlashReplica( - TiDB::DBInfoPtr db_info, - TiDB::TableInfoPtr latest_table_info, - ManageableStoragePtr storage) +void SchemaBuilder::applySetTiFlashReplicaOnPhysicalTable( + const TiDB::DBInfoPtr & db_info, + const TiDB::TableInfoPtr & latest_table_info, + const ManageableStoragePtr & storage) { if (storage->getTableInfo().replica_info.count == latest_table_info->replica_info.count) return; @@ -1238,6 +1264,75 @@ void SchemaBuilder::applySetTiFlashReplica( 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(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; + 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() { @@ -1306,7 +1401,9 @@ void SchemaBuilder::syncAllSchema() /// Rename if needed. applyRenameLogicalTable(db, table, storage); /// Update replica info if needed. - applySetTiFlashReplica(db, table, storage); + 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 8446765f74a..827203a682f 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -55,39 +55,44 @@ struct SchemaBuilder bool applyCreateSchema(DatabaseID schema_id); - void applyCreateSchema(TiDB::DBInfoPtr db_info); + void applyCreateSchema(const TiDB::DBInfoPtr & db_info); - void applyCreateTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyCreateTable(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyCreateLogicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info); + void applyCreateLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - void applyCreatePhysicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info); + void applyCreatePhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - void applyDropTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyDropTable(const TiDB::DBInfoPtr & db_info, TableID table_id); /// Parameter schema_name should be mapped. void applyDropPhysicalTable(const String & db_name, TableID table_id); - void applyPartitionDiff(TiDB::DBInfoPtr db_info, TableID table_id); + void applyPartitionDiff(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyPartitionDiff(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyPartitionDiff(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyAlterTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyAlterTable(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyAlterLogicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyAlterLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyAlterPhysicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyAlterPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyRenameTable(TiDB::DBInfoPtr new_db_info, TiDB::TableID table_id); + void applyRenameTable(const TiDB::DBInfoPtr & new_db_info, TiDB::TableID table_id); - void applyRenameLogicalTable(TiDB::DBInfoPtr new_db_info, TiDB::TableInfoPtr new_table_info, ManageableStoragePtr storage); + void applyRenameLogicalTable(const TiDB::DBInfoPtr & new_db_info, const TiDB::TableInfoPtr & new_table_info, const ManageableStoragePtr & storage); - void applyRenamePhysicalTable(TiDB::DBInfoPtr new_db_info, TiDB::TableInfo & new_table_info, ManageableStoragePtr storage); + void applyRenamePhysicalTable(const TiDB::DBInfoPtr & new_db_info, const TiDB::TableInfo & new_table_info, const ManageableStoragePtr & storage); void applyExchangeTablePartition(const SchemaDiff & diff); - void applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TableID table_id); - void applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + 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 fe0ecd59af0..72fd00678f7 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -94,11 +94,14 @@ enum class SchemaActionType : Int8 AlterTableStatsOptions = 58, 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 = 61, + MaxRecognizedType = 63, }; struct AffectedOption diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 8e18357031d..0d72f950add 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -542,7 +542,14 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/limit/", + "fill": 0, + "nullPointMode": "null", + "color": "#C4162A" + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, @@ -633,6 +640,13 @@ "intervalFactor": 1, "legendFormat": "{{instance}}", "refId": "K" + }, + { + "expr": "sum(tiflash_system_current_metric_MemoryCapacity{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "legendFormat": "limit-{{instance}}", + "exemplar": true, + "refId": "L", + "hide": false } ], "thresholds": [], @@ -701,15 +715,15 @@ "hiddenSeries": false, "id": 51, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -728,6 +742,12 @@ "alias": "total", "fill": 0, "lines": false + }, + { + "alias": "/limit/", + "fill": 0, + "nullPointMode": "null", + "color": "#C4162A" } ], "spaceLength": 10, @@ -742,6 +762,13 @@ "legendFormat": "{{instance}}", "refId": "A", "step": 40 + }, + { + "expr": "sum(tiflash_system_current_metric_LogicalCPUCores{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "legendFormat": "limit-{{instance}}", + "exemplar": true, + "refId": "B", + "intervalFactor": 1 } ], "thresholds": [], diff --git a/tests/fullstack-test/expr/bitshift_operator.test b/tests/fullstack-test/expr/bitshift_operator.test new file mode 100644 index 00000000000..0d55a1b56a9 --- /dev/null +++ b/tests/fullstack-test/expr/bitshift_operator.test @@ -0,0 +1,43 @@ +# 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. + +mysql> drop table if exists test.t; +mysql> create table test.t (a int); +mysql> alter table test.t set tiflash replica 1; +mysql> insert into test.t values(-1); + +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set @@session.tidb_isolation_read_engines = "tiflash"; select a>>0 as v1, a>>64 as v2, a>>10 as v3 from test.t; ++----------------------+------+-------------------+ +| v1 | v2 | v3 | ++----------------------+------+-------------------+ +| 18446744073709551615 | 0 | 18014398509481983 | ++----------------------+------+-------------------+ + +mysql> set tidb_enforce_mpp=1; set @@session.tidb_isolation_read_engines = "tiflash"; select a from test.t where a>>100000=0; ++------+ +| a | ++------+ +| -1 | ++------+ + +mysql> set tidb_enforce_mpp=1; set @@session.tidb_isolation_read_engines = "tiflash"; select a from test.t where a>>63=1; ++------+ +| a | ++------+ +| -1 | ++------+ + +mysql> drop table if exists test.t diff --git a/tests/fullstack-test/expr/duration_pushdown.test b/tests/fullstack-test/expr/duration_pushdown.test index 63106fa1788..442a708a802 100644 --- a/tests/fullstack-test/expr/duration_pushdown.test +++ b/tests/fullstack-test/expr/duration_pushdown.test @@ -106,6 +106,14 @@ mysql> use test; set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflas # | 123500 | # +----------------+ +mysql> use test; set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select time_to_sec(a) from t; ++----------------+ +| time_to_sec(a) | ++----------------+ +| 2520610 | +| -2520610 | ++----------------+ + mysql> drop table if exists test.time_test; mysql> create table test.time_test(id int(11),v1 time(3) not null, v2 time(3)); diff --git a/tests/fullstack-test/expr/reverse.test b/tests/fullstack-test/expr/reverse.test new file mode 100644 index 00000000000..9195adf2b7d --- /dev/null +++ b/tests/fullstack-test/expr/reverse.test @@ -0,0 +1,44 @@ +# 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. + +mysql> drop table if exists test.t; +mysql> create table if not exists test.t(a varchar(256)); + + +mysql> insert into test.t values('one week’s time test'); +mysql> insert into test.t values('abc测试def'); +mysql> insert into test.t values('abcテストabc'); +mysql> insert into test.t values('ѐёђѓєѕіїјљњћќѝўџ'); +mysql> insert into test.t values('+ѐ-ё*ђ/ѓ!є@ѕ#і@ї%ј……љ&њ(ћ)ќ¥ѝ#ў@џ!^'); +mysql> insert into test.t values('αβγδεζηθικλμνξοπρστυφχψωσ'); +mysql> insert into test.t values('▲α▼βγ➨δε☎ζη✂θι€κλ♫μν✓ξο✚πρ℉στ♥υφ♖χψ♘ω★σ✕'); +mysql> insert into test.t values('թփձջրչճժծքոեռտըւիօպասդֆգհյկլխզղցվբնմշ'); +mysql> insert into test.t values(NULL); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select reverse(a) from test.t; ++-------------------------------------------------------------------------------------------------+ +| reverse(a) | ++-------------------------------------------------------------------------------------------------+ +| tset emit s’keew eno | +| fed试测cba | +| cbaトステcba | +| џўѝќћњљјїіѕєѓђёѐ | +| ^!џ@ў#ѝ¥ќ)ћ(њ&љ……ј%ї@і#ѕ@є!ѓ/ђ*ё-ѐ+ | +| σωψχφυτσρποξνμλκιθηζεδγβα | +| ✕σ★ω♘ψχ♖φυ♥τσ℉ρπ✚οξ✓νμ♫λκ€ιθ✂ηζ☎εδ➨γβ▼α▲ | +| շմնբվցղզխլկյհգֆդսապօիւըտռեոքծժճչրջձփթ | +| NULL | ++-------------------------------------------------------------------------------------------------+ diff --git a/tests/fullstack-test/mpp/issue_2471.test b/tests/fullstack-test/mpp/issue_2471.test index 497ce605893..9966eaadec3 100644 --- a/tests/fullstack-test/mpp/issue_2471.test +++ b/tests/fullstack-test/mpp/issue_2471.test @@ -35,15 +35,7 @@ mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_bro => DBGInvoke __enable_fail_point(exception_in_creating_set_input_stream) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_broadcast_cartesian_join=2; select * from a as t1 left join a as t2 on t1.id = t2.id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_in_creating_set_input_stream) diff --git a/tests/fullstack-test/mpp/mpp_fail.test b/tests/fullstack-test/mpp/mpp_fail.test index e03c6150be6..0e272c0b621 100644 --- a/tests/fullstack-test/mpp/mpp_fail.test +++ b/tests/fullstack-test/mpp/mpp_fail.test @@ -71,44 +71,20 @@ ERROR 1105 (HY000) at line 1: DB::Exception: Fail point FailPoints::exception_be ## exception during mpp run non root task => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception,, e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) ## exception during mpp run root task => DBGInvoke __enable_fail_point(exception_during_mpp_root_task_run) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_root_task_run) ## exception during mpp write err to tunnel => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __enable_fail_point(exception_during_mpp_write_err_to_tunnel) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __disable_fail_point(exception_during_mpp_write_err_to_tunnel) @@ -116,14 +92,7 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __enable_fail_point(exception_during_mpp_close_tunnel) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception,, e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __disable_fail_point(exception_during_mpp_close_tunnel) @@ -156,16 +125,7 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText ## ensure build1, build2-probe1, probe2 in the CreatingSets, test the bug where build1 throw exception but not change the build state, thus block the build2-probe1, at last this query hangs. => DBGInvoke __enable_fail_point(exception_mpp_hash_build) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count=0; set @@tidb_broadcast_join_threshold_size=0; select t1.id from test.t t1 join test.t t2 on t1.id = t2.id and t1.id <2 join (select id from test.t group by id) t3 on t2.id=t3.id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_mpp_hash_build) # Clean up. diff --git a/tests/fullstack-test2/ddl/alter_table_tiflash_replica_and_mode.test b/tests/fullstack-test2/ddl/alter_table_tiflash_replica_and_mode.test new file mode 100644 index 00000000000..5e43936379b --- /dev/null +++ b/tests/fullstack-test2/ddl/alter_table_tiflash_replica_and_mode.test @@ -0,0 +1,89 @@ +# 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 tiflash replica for normal case +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 + +func> wait_table test t + +>> DBGInvoke get_tiflash_replica_count("test", "t") +┌─get_tiflash_replica_count(test, t)─┐ +│ 1 │ +└────────────────────────────────────┘ + +# test tiflash mode in normal mode +>> DBGInvoke get_tiflash_mode("test", "t") +┌─get_tiflash_mode(test, t)─┐ +│ │ +└───────────────────────────┘ + +mysql> alter table test.t set tiflash mode fast + +>> DBGInvoke __refresh_schemas() + +# test tiflash mode in fast mode +>> DBGInvoke get_tiflash_mode("test", "t") +┌─get_tiflash_mode(test, t)───┐ +│ fast │ +└─────────────────────────────┘ + +# test replica for partition tables +mysql> drop table if exists test.t +mysql> create table test.t (x int) partition by range (x) (partition p0 values less than (5), partition p1 values less than (10)); +mysql> alter table test.t set tiflash mode fast +mysql> alter table test.t set tiflash replica 1 + +func> wait_table test t + +>> DBGInvoke get_tiflash_replica_count("test", "t") +┌─get_tiflash_replica_count(test, t)─┐ +│ 1 │ +└────────────────────────────────────┘ + +>> DBGInvoke get_tiflash_mode("test", "t") +┌─get_tiflash_mode(test, t)──────────┐ +│ fast │ +└────────────────────────────────────┘ + +>> DBGInvoke get_partition_tables_tiflash_replica_count("test", "t") +┌─get_partition_tables_tiflash_replica_count(test, t)─┐ +│ 1/1/ │ +└─────────────────────────────────────────────────────┘ + +# test tiflash mode for partition tables +>> DBGInvoke get_partition_tables_tiflash_mode("test", "t") +┌─get_partition_tables_tiflash_mode(test, t)─┐ +│ fast/fast/ │ +└────────────────────────────────────────────┘ + +# test replica for add partition tables after set replica +mysql> alter table test.t add partition (partition p2 values less than (2010)); + +>> DBGInvoke __refresh_schemas() + +>> DBGInvoke get_partition_tables_tiflash_replica_count("test", "t") +┌─get_partition_tables_tiflash_replica_count(test, t)─┐ +│ 1/1/1/ │ +└─────────────────────────────────────────────────────┘ + +# test tiflash mode for add partition tables after set replica +>> DBGInvoke get_partition_tables_tiflash_mode("test", "t") +┌─get_partition_tables_tiflash_mode(test, t)─┐ +│ fast/fast/fast/ │ +└────────────────────────────────────────────┘ + + + diff --git a/tests/fullstack-test2/ddl/alter_tiflash_mode.test b/tests/fullstack-test2/ddl/alter_tiflash_mode.test new file mode 100644 index 00000000000..c9f3ef488c4 --- /dev/null +++ b/tests/fullstack-test2/ddl/alter_tiflash_mode.test @@ -0,0 +1,48 @@ +# 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. + +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 + +# check default mode of tiflash table +mysql> select table_schema,table_name,replica_count,available,table_mode from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------+-----------+ +| table_schema | table_name | replica_count | available | table_mode| ++--------------+------------+---------------+-----------+-----------+ +| test | t | 1 | 1 | NORMAL | ++--------------+------------+---------------+-----------+-----------+ + +# check change mode + +mysql> alter table test.t set tiflash mode fast +mysql> select table_schema,table_name,replica_count,available,table_mode from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------+-----------+ +| table_schema | table_name | replica_count | available | table_mode| ++--------------+------------+---------------+-----------+-----------+ +| test | t | 1 | 1 | FAST | ++--------------+------------+---------------+-----------+-----------+ + +# check change mode +mysql> alter table test.t set tiflash mode normal +mysql> select table_schema,table_name,replica_count,available,table_mode from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------+-----------+ +| table_schema | table_name | replica_count | available | table_mode| ++--------------+------------+---------------+-----------+-----------+ +| test | t | 1 | 1 | NORMAL | ++--------------+------------+---------------+-----------+-----------+ + +mysql> drop table if exists test.t \ No newline at end of file diff --git a/tests/tidb-ci/new_collation_fullstack/expr.test b/tests/tidb-ci/new_collation_fullstack/expr.test index 15ada0f335c..1e2135c4f2d 100644 --- a/tests/tidb-ci/new_collation_fullstack/expr.test +++ b/tests/tidb-ci/new_collation_fullstack/expr.test @@ -35,6 +35,13 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_s | 2 | abc | +------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value1 from test.t where value1 = 'abc '; ++------+-------+ +| id | value1| ++------+-------+ +| 1 | abc | +| 2 | abc | ++------+-------+ mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value from test.t where value like 'aB%'; +------+-------+ @@ -62,6 +69,13 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_s | 3 | def | +------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value1 from test.t where value1 = 'def '; ++------+-------+ +| id | value1| ++------+-------+ +| 3 | def | ++------+-------+ + mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value1 from test.t where value1 in ('Abc','def'); +------+-------+ | id | value1|