diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index f7d881f0d06..993c455d854 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -77,6 +77,7 @@ namespace DB M(skip_check_segment_update) \ M(force_set_page_file_write_errno) \ M(force_split_io_size_4k) \ + M(force_set_num_regions_for_table) \ M(minimum_block_size_for_cross_join) \ M(random_exception_after_dt_write_done) \ M(random_slow_page_storage_write) \ diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index 6b2cace7951..aa3c4cfa814 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -64,17 +64,47 @@ MockTiDB::MockTiDB() databases["default"] = 0; } -TablePtr MockTiDB::dropTableInternal(Context & context, const String & database_name, const String & table_name, bool drop_regions) +TablePtr MockTiDB::dropTableByNameImpl(Context & context, const String & database_name, const String & table_name, bool drop_regions) { String qualified_name = database_name + "." + table_name; auto it_by_name = tables_by_name.find(qualified_name); if (it_by_name == tables_by_name.end()) return nullptr; + auto table = it_by_name->second; + dropTableInternal(context, table, drop_regions); + + tables_by_name.erase(it_by_name); + return table; +} + +TablePtr MockTiDB::dropTableByIdImpl(Context & context, const TableID table_id, bool drop_regions) +{ + auto iter = tables_by_id.find(table_id); + if (iter == tables_by_id.end()) + return nullptr; + + auto table = iter->second; + dropTableInternal(context, table, drop_regions); + + // erase from `tables_by_name` + for (auto iter_by_name = tables_by_name.begin(); iter_by_name != tables_by_name.end(); /* empty */) + { + if (table != iter_by_name->second) + { + ++iter_by_name; + continue; + } + LOG_INFO(Logger::get(), "removing table from MockTiDB, name={} table_id={}", iter_by_name->first, table_id); + iter_by_name = tables_by_name.erase(iter_by_name); + } + return table; +} +TablePtr MockTiDB::dropTableInternal(Context & context, const TablePtr & table, bool drop_regions) +{ auto & kvstore = context.getTMTContext().getKVStore(); auto & region_table = context.getTMTContext().getRegionTable(); - auto table = it_by_name->second; if (table->isPartitionTable()) { for (const auto & partition : table->table_info.partition.definitions) @@ -90,15 +120,12 @@ TablePtr MockTiDB::dropTableInternal(Context & context, const String & database_ } tables_by_id.erase(table->id()); - tables_by_name.erase(it_by_name); - if (drop_regions) { for (auto & e : region_table.getRegionsByTable(NullspaceID, table->id())) kvstore->mockRemoveRegion(e.first, region_table); region_table.removeTable(NullspaceID, table->id()); } - return table; } @@ -113,7 +140,7 @@ void MockTiDB::dropDB(Context & context, const String & database_name, bool drop }); for (const auto & table_name : table_names) - dropTableInternal(context, database_name, table_name, drop_regions); + dropTableByNameImpl(context, database_name, table_name, drop_regions); version++; @@ -132,8 +159,25 @@ void MockTiDB::dropDB(Context & context, const String & database_name, bool drop void MockTiDB::dropTable(Context & context, const String & database_name, const String & table_name, bool drop_regions) { std::lock_guard lock(tables_mutex); + auto table = dropTableByNameImpl(context, database_name, table_name, drop_regions); + if (!table) + return; + + version++; - auto table = dropTableInternal(context, database_name, table_name, drop_regions); + SchemaDiff diff; + diff.type = SchemaActionType::DropTable; + diff.schema_id = table->database_id; + diff.table_id = table->id(); + diff.version = version; + version_diff[version] = diff; +} + +void MockTiDB::dropTableById(Context & context, const TableID & table_id, bool drop_regions) +{ + std::lock_guard lock(tables_mutex); + + auto table = dropTableByIdImpl(context, table_id, drop_regions); if (!table) return; @@ -273,13 +317,15 @@ TableID MockTiDB::newTable( return addTable(database_name, std::move(*table_info)); } -int MockTiDB::newTables( +std::vector MockTiDB::newTables( const String & database_name, const std::vector> & tables, Timestamp tso, const String & engine_type) { - std::lock_guard lock(tables_mutex); + std::scoped_lock lock(tables_mutex); + std::vector table_ids; + table_ids.reserve(tables.size()); if (databases.find(database_name) == databases.end()) { throw Exception("MockTiDB not found db: " + database_name, ErrorCodes::LOGICAL_ERROR); @@ -300,7 +346,8 @@ int MockTiDB::newTables( table_info.id = table_id_allocator++; table_info.update_timestamp = tso; - auto table = std::make_shared(database_name, databases[database_name], table_info.name, std::move(table_info)); + auto table + = std::make_shared
(database_name, databases[database_name], table_info.name, std::move(table_info)); tables_by_id.emplace(table->table_info.id, table); tables_by_name.emplace(qualified_name, table); @@ -310,6 +357,8 @@ int MockTiDB::newTables( opt.old_schema_id = table->database_id; opt.old_table_id = table->id(); diff.affected_opts.push_back(std::move(opt)); + + table_ids.push_back(table->id()); } if (diff.affected_opts.empty()) @@ -318,7 +367,8 @@ int MockTiDB::newTables( diff.schema_id = diff.affected_opts[0].schema_id; diff.version = version; version_diff[version] = diff; - return 0; + + return table_ids; } TableID MockTiDB::addTable(const String & database_name, TiDB::TableInfo && table_info) diff --git a/dbms/src/Debug/MockTiDB.h b/dbms/src/Debug/MockTiDB.h index ce9fc18b764..82f3f6e8291 100644 --- a/dbms/src/Debug/MockTiDB.h +++ b/dbms/src/Debug/MockTiDB.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -82,7 +83,7 @@ class MockTiDB : public ext::Singleton const String & handle_pk_name, const String & engine_type); - int newTables( + std::vector newTables( const String & database_name, const std::vector> & tables, Timestamp tso, @@ -104,6 +105,7 @@ class MockTiDB : public ext::Singleton void dropPartition(const String & database_name, const String & table_name, TableID partition_id); void dropTable(Context & context, const String & database_name, const String & table_name, bool drop_regions); + void dropTableById(Context & context, const TableID & table_id, bool drop_regions); void dropDB(Context & context, const String & database_name, bool drop_regions); @@ -151,7 +153,9 @@ class MockTiDB : public ext::Singleton private: TableID newPartitionImpl(const TablePtr & logical_table, TableID partition_id, const String & partition_name, Timestamp tso, bool is_add_part); - TablePtr dropTableInternal(Context & context, const String & database_name, const String & table_name, bool drop_regions); + TablePtr dropTableByNameImpl(Context & context, const String & database_name, const String & table_name, bool drop_regions); + TablePtr dropTableByIdImpl(Context & context, TableID table_id, bool drop_regions); + TablePtr dropTableInternal(Context & context, const TablePtr & table, bool drop_regions); TablePtr getTableByNameInternal(const String & database_name, const String & table_name); TablePtr getTableByID(TableID table_id); diff --git a/dbms/src/Debug/dbgFuncSchema.cpp b/dbms/src/Debug/dbgFuncSchema.cpp index 090cfc4c53b..a3bf9e17a52 100644 --- a/dbms/src/Debug/dbgFuncSchema.cpp +++ b/dbms/src/Debug/dbgFuncSchema.cpp @@ -86,16 +86,20 @@ void dbgFuncRefreshSchemas(Context & context, const ASTs &, DBGInvoker::Printer // Trigger gc on all databases / tables. // Usage: -// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point])" +// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point, ignore_remain_regions])" void dbgFuncGcSchemas(Context & context, const ASTs & args, DBGInvoker::Printer output) { auto & service = context.getSchemaSyncService(); Timestamp gc_safe_point = 0; + bool ignore_remain_regions = false; if (args.empty()) gc_safe_point = PDClientHelper::getGCSafePointWithRetry(context.getTMTContext().getPDClient()); - else + if (!args.empty()) gc_safe_point = safeGet(typeid_cast(*args[0]).value); - service->gc(gc_safe_point, NullspaceID); + if (args.size() >= 2) + ignore_remain_regions = safeGet(typeid_cast(*args[1]).value) == "true"; + // Note that only call it in tests, we need to ignore remain regions + service->gcImpl(gc_safe_point, NullspaceID, ignore_remain_regions); output("schemas gc done"); } @@ -139,4 +143,4 @@ void dbgFuncIsTombstone(Context & context, const ASTs & args, DBGInvoker::Printe } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchema.h b/dbms/src/Debug/dbgFuncSchema.h index a92dc2418ab..46c3e5e8450 100644 --- a/dbms/src/Debug/dbgFuncSchema.h +++ b/dbms/src/Debug/dbgFuncSchema.h @@ -34,7 +34,7 @@ void dbgFuncRefreshSchemas(Context & context, const ASTs & args, DBGInvoker::Pri // Trigger gc on all databases / tables. // Usage: -// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point])" +// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point, ignore_remain_regions])" void dbgFuncGcSchemas(Context & context, const ASTs & args, DBGInvoker::Printer output); // Reset schemas. diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 6603755bd92..e2f28c02870 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -140,7 +140,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::LTDecimal, "less"}, {tipb::ScalarFuncSig::LTTime, "less"}, {tipb::ScalarFuncSig::LTDuration, "less"}, - {tipb::ScalarFuncSig::LTJson, "less"}, + //{tipb::ScalarFuncSig::LTJson, "less"}, {tipb::ScalarFuncSig::LEInt, "lessOrEquals"}, {tipb::ScalarFuncSig::LEReal, "lessOrEquals"}, @@ -148,7 +148,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::LEDecimal, "lessOrEquals"}, {tipb::ScalarFuncSig::LETime, "lessOrEquals"}, {tipb::ScalarFuncSig::LEDuration, "lessOrEquals"}, - {tipb::ScalarFuncSig::LEJson, "lessOrEquals"}, + //{tipb::ScalarFuncSig::LEJson, "lessOrEquals"}, {tipb::ScalarFuncSig::GTInt, "greater"}, {tipb::ScalarFuncSig::GTReal, "greater"}, @@ -156,7 +156,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::GTDecimal, "greater"}, {tipb::ScalarFuncSig::GTTime, "greater"}, {tipb::ScalarFuncSig::GTDuration, "greater"}, - {tipb::ScalarFuncSig::GTJson, "greater"}, + //{tipb::ScalarFuncSig::GTJson, "greater"}, {tipb::ScalarFuncSig::GreatestInt, "tidbGreatest"}, {tipb::ScalarFuncSig::GreatestReal, "tidbGreatest"}, @@ -179,7 +179,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::GEDecimal, "greaterOrEquals"}, {tipb::ScalarFuncSig::GETime, "greaterOrEquals"}, {tipb::ScalarFuncSig::GEDuration, "greaterOrEquals"}, - {tipb::ScalarFuncSig::GEJson, "greaterOrEquals"}, + //{tipb::ScalarFuncSig::GEJson, "greaterOrEquals"}, {tipb::ScalarFuncSig::EQInt, "equals"}, {tipb::ScalarFuncSig::EQReal, "equals"}, @@ -187,7 +187,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::EQDecimal, "equals"}, {tipb::ScalarFuncSig::EQTime, "equals"}, {tipb::ScalarFuncSig::EQDuration, "equals"}, - {tipb::ScalarFuncSig::EQJson, "equals"}, + //{tipb::ScalarFuncSig::EQJson, "equals"}, {tipb::ScalarFuncSig::NEInt, "notEquals"}, {tipb::ScalarFuncSig::NEReal, "notEquals"}, @@ -195,7 +195,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::NEDecimal, "notEquals"}, {tipb::ScalarFuncSig::NETime, "notEquals"}, {tipb::ScalarFuncSig::NEDuration, "notEquals"}, - {tipb::ScalarFuncSig::NEJson, "notEquals"}, + //{tipb::ScalarFuncSig::NEJson, "notEquals"}, //{tipb::ScalarFuncSig::NullEQInt, "cast"}, //{tipb::ScalarFuncSig::NullEQReal, "cast"}, @@ -312,7 +312,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::StringIsNull, "isNull"}, {tipb::ScalarFuncSig::TimeIsNull, "isNull"}, {tipb::ScalarFuncSig::IntIsNull, "isNull"}, - {tipb::ScalarFuncSig::JsonIsNull, "isNull"}, + //{tipb::ScalarFuncSig::JsonIsNull, "isNull"}, {tipb::ScalarFuncSig::BitAndSig, "bitAnd"}, {tipb::ScalarFuncSig::BitOrSig, "bitOr"}, @@ -353,7 +353,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::InDecimal, "tidbIn"}, {tipb::ScalarFuncSig::InTime, "tidbIn"}, {tipb::ScalarFuncSig::InDuration, "tidbIn"}, - {tipb::ScalarFuncSig::InJson, "tidbIn"}, + //{tipb::ScalarFuncSig::InJson, "tidbIn"}, {tipb::ScalarFuncSig::IfNullInt, "ifNull"}, {tipb::ScalarFuncSig::IfNullReal, "ifNull"}, diff --git a/dbms/src/Functions/FunctionsNull.h b/dbms/src/Functions/FunctionsNull.h index 7e37305d300..046beb380fe 100644 --- a/dbms/src/Functions/FunctionsNull.h +++ b/dbms/src/Functions/FunctionsNull.h @@ -64,7 +64,7 @@ class FunctionCoalesce : public IFunction public: static constexpr auto name = "coalesce"; static FunctionPtr create(const Context & context); - FunctionCoalesce(const Context & context) + explicit FunctionCoalesce(const Context & context) : context(context) {} diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp index 842ffd3f1df..d856872bfe4 100644 --- a/dbms/src/Functions/FunctionsString.cpp +++ b/dbms/src/Functions/FunctionsString.cpp @@ -13,6 +13,8 @@ // limitations under the License. #include +#include +#include #include #include #include @@ -4127,9 +4129,11 @@ class FunctionASCII : public IFunction std::string getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() != 1) + if unlikely (arguments.size() != 1) throw Exception( fmt::format("Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -4140,28 +4144,25 @@ class FunctionASCII : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const IColumn * c0_col = block.getByPosition(arguments[0]).column.get(); - const auto * c0_const = checkAndGetColumn(c0_col); const auto * c0_string = checkAndGetColumn(c0_col); + if unlikely (c0_string == nullptr) + throw Exception( + fmt::format("Illegal argument of function {}", getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - Field res_field; - int val_num = c0_col->size(); + auto val_num = static_cast(c0_col->size()); auto col_res = ColumnInt64::create(); - col_res->reserve(val_num); - if (c0_const == nullptr && c0_string == nullptr) - throw Exception(fmt::format("Illegal argument of function {}", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ColumnInt64::Container & data = col_res->getData(); + data.resize(val_num); - for (int i = 0; i < val_num; i++) - { - c0_col->get(i, res_field); - String handled_str = res_field.get(); - Int64 res = handled_str.empty() ? 0 : static_cast(handled_str[0]); - col_res->insert(res); - } + const auto & chars = c0_string->getChars(); + const auto & offsets = c0_string->getOffsets(); + + for (ssize_t i = 0; i < val_num; i++) + data[i] = chars[offsets[i - 1]]; block.getByPosition(result).column = std::move(col_res); } - -private: }; class FunctionLength : public IFunction @@ -4178,9 +4179,11 @@ class FunctionLength : public IFunction std::string getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() != 1) + if unlikely (arguments.size() != 1) throw Exception( fmt::format("Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -4191,22 +4194,21 @@ class FunctionLength : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const IColumn * c0_col = block.getByPosition(arguments[0]).column.get(); - const auto * c0_const = checkAndGetColumn(c0_col); const auto * c0_string = checkAndGetColumn(c0_col); + if unlikely (c0_string == nullptr) + throw Exception( + fmt::format("Illegal argument of function {}", getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - Field res_field; - int val_num = c0_col->size(); + auto val_num = static_cast(c0_col->size()); auto col_res = ColumnInt64::create(); - col_res->reserve(val_num); - if (c0_const == nullptr && c0_string == nullptr) - throw Exception(fmt::format("Illegal argument of function {}", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ColumnInt64::Container & data = col_res->getData(); + data.resize(val_num); - for (int i = 0; i < val_num; i++) - { - c0_col->get(i, res_field); - String handled_str = res_field.get(); - col_res->insert(static_cast(handled_str.size())); - } + const auto & offsets = c0_string->getOffsets(); + + for (ssize_t i = 0; i < val_num; i++) + data[i] = offsets[i] - offsets[i - 1] - 1; block.getByPosition(result).column = std::move(col_res); } diff --git a/dbms/src/Functions/FunctionsTiDBConversion.h b/dbms/src/Functions/FunctionsTiDBConversion.h index 023a8b26698..d3de8e6bf00 100755 --- a/dbms/src/Functions/FunctionsTiDBConversion.h +++ b/dbms/src/Functions/FunctionsTiDBConversion.h @@ -951,12 +951,13 @@ struct TiDBConvertToDecimal } else if (v_scale > scale) { - const bool need_to_round = ((value < 0 ? -value : value) % scale_mul) >= (scale_mul / 2); + const bool neg = (value < 0); + const bool need_to_round = ((neg ? -value : value) % scale_mul) >= (scale_mul / 2); auto old_value = value; value /= scale_mul; if (need_to_round) { - if (value < 0) + if (neg) --value; else ++value; diff --git a/dbms/src/Functions/GatherUtils/Algorithms.h b/dbms/src/Functions/GatherUtils/Algorithms.h index 3b63e6ae328..659238caa1c 100644 --- a/dbms/src/Functions/GatherUtils/Algorithms.h +++ b/dbms/src/Functions/GatherUtils/Algorithms.h @@ -187,7 +187,7 @@ void concat(const std::vector> & array_sources, Si size_t sources_num = array_sources.size(); std::vector is_const(sources_num); - auto checkAndGetSizeToReserve = [](auto source, IArraySource * array_source) { + auto check_and_get_size_to_reserve = [](auto source, IArraySource * array_source) { if (source == nullptr) throw Exception("Concat function expected " + demangle(typeid(Source).name()) + " or " + demangle(typeid(ConstSource).name()) + " but got " @@ -199,17 +199,18 @@ void concat(const std::vector> & array_sources, Si size_t size_to_reserve = 0; for (auto i : ext::range(0, sources_num)) { - auto & source = array_sources[i]; + const auto & source = array_sources[i]; is_const[i] = source->isConst(); if (is_const[i]) - size_to_reserve += checkAndGetSizeToReserve(typeid_cast *>(source.get()), source.get()); + size_to_reserve + += check_and_get_size_to_reserve(typeid_cast *>(source.get()), source.get()); else - size_to_reserve += checkAndGetSizeToReserve(typeid_cast(source.get()), source.get()); + size_to_reserve += check_and_get_size_to_reserve(typeid_cast(source.get()), source.get()); } sink.reserve(size_to_reserve); - auto writeNext = [&sink](auto source) { + auto write_next = [&sink](auto source) { writeSlice(source->getWhole(), sink); source->next(); }; @@ -218,11 +219,11 @@ void concat(const std::vector> & array_sources, Si { for (auto i : ext::range(0, sources_num)) { - auto & source = array_sources[i]; + const auto & source = array_sources[i]; if (is_const[i]) - writeNext(static_cast *>(source.get())); + write_next(static_cast *>(source.get())); else - writeNext(static_cast(source.get())); + write_next(static_cast(source.get())); } sink.next(); } @@ -383,11 +384,11 @@ void NO_INLINE pad(SourceA && src, SourceB && padding, Sink && sink, ssize_t len size_t left = static_cast(length) - slice.size; if (is_left) { - StringSource::Slice padSlice = padding.getWhole(); - while (left > padSlice.size && padSlice.size != 0) + StringSource::Slice pad_slice = padding.getWhole(); + while (left > pad_slice.size && pad_slice.size != 0) { - writeSlice(padSlice, sink); - left -= padSlice.size; + writeSlice(pad_slice, sink); + left -= pad_slice.size; } writeSlice(padding.getSliceFromLeft(0, left), sink); @@ -396,11 +397,11 @@ void NO_INLINE pad(SourceA && src, SourceB && padding, Sink && sink, ssize_t len else { writeSlice(slice, sink); - StringSource::Slice padSlice = padding.getWhole(); - while (left > padSlice.size && padSlice.size != 0) + StringSource::Slice pad_slice = padding.getWhole(); + while (left > pad_slice.size && pad_slice.size != 0) { - writeSlice(padSlice, sink); - left -= padSlice.size; + writeSlice(pad_slice, sink); + left -= pad_slice.size; } writeSlice(padding.getSliceFromLeft(0, left), sink); diff --git a/dbms/src/Functions/tests/bench_function_ilike.cpp b/dbms/src/Functions/tests/bench_function_ilike.cpp deleted file mode 100644 index 173b5449a0e..00000000000 --- a/dbms/src/Functions/tests/bench_function_ilike.cpp +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include - -/// this is a hack, include the cpp file so we can test MatchImpl directly -#include -#include // NOLINT - -namespace DB -{ -namespace tests -{ - -constexpr size_t data_num = 500000; - -class IlikeBench : public benchmark::Fixture -{ -public: - using ColStringType = typename TypeTraits::FieldType; - using ColUInt8Type = typename TypeTraits::FieldType; - - ColumnWithTypeAndName escape = createConstColumn(1, static_cast('\\')); - - ColumnsWithTypeAndName data1{ - toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - escape}; - ColumnsWithTypeAndName data2{ - toVec("col0", std::vector(data_num, "AAAAAAAAAAAAAAAAA")), - toVec("col1", std::vector(data_num, "AAAAAAAAAAAAAAAAA")), - escape}; - ColumnsWithTypeAndName data3{ - toVec("col0", std::vector(data_num, "aAaAaAaAaAaAaAaAa")), - toVec("col1", std::vector(data_num, "aAaAaAaAaAaAaAaAa")), - escape}; - ColumnsWithTypeAndName data4{ - toVec("col0", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), - toVec("col1", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), - escape}; - ColumnsWithTypeAndName data5{ - toVec("col0", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), - toVec("col1", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), - escape}; - - void SetUp(const benchmark::State &) override {} -}; - -class LikeBench : public benchmark::Fixture -{ -public: - using ColStringType = typename TypeTraits::FieldType; - using ColUInt8Type = typename TypeTraits::FieldType; - - ColumnWithTypeAndName escape = createConstColumn(1, static_cast('\\')); - - ColumnsWithTypeAndName lower_data11{toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa"))}; - ColumnsWithTypeAndName lower_data12{toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa"))}; - - ColumnsWithTypeAndName lower_data21{toVec("col0", std::vector(data_num, "AAAAAAAAAAAAAAAAA"))}; - ColumnsWithTypeAndName lower_data22{toVec("col1", std::vector(data_num, "AAAAAAAAAAAAAAAAA"))}; - - ColumnsWithTypeAndName lower_data31{toVec("col0", std::vector(data_num, "aAaAaAaAaAaAaAaAa"))}; - ColumnsWithTypeAndName lower_data32{toVec("col1", std::vector(data_num, "aAaAaAaAaAaAaAaAa"))}; - - ColumnsWithTypeAndName lower_data41{toVec("col0", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯"))}; - ColumnsWithTypeAndName lower_data42{toVec("col1", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯"))}; - - ColumnsWithTypeAndName lower_data51{toVec("col0", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯"))}; - ColumnsWithTypeAndName lower_data52{toVec("col1", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯"))}; - - ColumnsWithTypeAndName like_data1{ - toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - escape}; - ColumnsWithTypeAndName like_data2{ - toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - escape}; - ColumnsWithTypeAndName like_data3{ - toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), - escape}; - ColumnsWithTypeAndName like_data4{ - toVec("col0", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), - toVec("col1", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), - escape}; - ColumnsWithTypeAndName like_data5{ - toVec("col0", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), - toVec("col1", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), - escape}; - - void SetUp(const benchmark::State &) override {} -}; - -BENCHMARK_DEFINE_F(IlikeBench, ilike) -(benchmark::State & state) -try -{ - FunctionIlike3Args function_ilike; - TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_BIN); - function_ilike.setCollator(collator); - std::vector blocks{Block(data1), Block(data2), Block(data3), Block(data4), Block(data5)}; - for (auto & block : blocks) - block.insert({nullptr, std::make_shared>(), "res"}); - ColumnNumbers arguments{0, 1, 2}; - for (auto _ : state) - { - for (auto & block : blocks) - function_ilike.executeImpl(block, arguments, 3); - } -} -CATCH -BENCHMARK_REGISTER_F(IlikeBench, ilike)->Iterations(10); - -BENCHMARK_DEFINE_F(LikeBench, like) -(benchmark::State & state) -try -{ - FunctionLowerUTF8 function_lower; - FunctionLike function_like; - TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_BIN); - function_like.setCollator(collator); - std::vector lower_blocks{ - Block(lower_data11), - Block(lower_data21), - Block(lower_data31), - Block(lower_data41), - Block(lower_data51), - Block(lower_data12), - Block(lower_data22), - Block(lower_data32), - Block(lower_data42), - Block(lower_data52)}; - std::vector like_blocks{Block(like_data1), Block(like_data2), Block(like_data3), Block(like_data4), Block(like_data5)}; - - for (auto & block : lower_blocks) - block.insert({nullptr, std::make_shared(), "res"}); - for (auto & block : like_blocks) - block.insert({nullptr, std::make_shared>(), "res"}); - - ColumnNumbers lower_arguments{0, 1}; - ColumnNumbers like_arguments{0, 1, 2}; - for (auto _ : state) - { - for (auto & block : lower_blocks) - function_lower.executeImpl(block, lower_arguments, 1); - for (auto & block : like_blocks) - function_like.executeImpl(block, like_arguments, 3); - } -} -CATCH -BENCHMARK_REGISTER_F(LikeBench, like)->Iterations(10); - -} // namespace tests -} // namespace DB diff --git a/dbms/src/Functions/tests/bench_function_string.cpp b/dbms/src/Functions/tests/bench_function_string.cpp new file mode 100644 index 00000000000..16dc6df84f7 --- /dev/null +++ b/dbms/src/Functions/tests/bench_function_string.cpp @@ -0,0 +1,362 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include + +/// this is a hack, include the cpp file so we can test functions directly +#include // NOLINT +#include // NOLINT + +namespace DB +{ +namespace tests +{ + +constexpr size_t data_num = 500000; + +class IlikeBench : public benchmark::Fixture +{ +public: + using ColStringType = typename TypeTraits::FieldType; + using ColUInt8Type = typename TypeTraits::FieldType; + + ColumnWithTypeAndName escape = createConstColumn(1, static_cast('\\')); + + ColumnsWithTypeAndName data1{ + toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + escape}; + ColumnsWithTypeAndName data2{ + toVec("col0", std::vector(data_num, "AAAAAAAAAAAAAAAAA")), + toVec("col1", std::vector(data_num, "AAAAAAAAAAAAAAAAA")), + escape}; + ColumnsWithTypeAndName data3{ + toVec("col0", std::vector(data_num, "aAaAaAaAaAaAaAaAa")), + toVec("col1", std::vector(data_num, "aAaAaAaAaAaAaAaAa")), + escape}; + ColumnsWithTypeAndName data4{ + toVec("col0", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), + toVec("col1", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), + escape}; + ColumnsWithTypeAndName data5{ + toVec("col0", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), + toVec("col1", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), + escape}; + + void SetUp(const benchmark::State &) override {} +}; + +class LikeBench : public benchmark::Fixture +{ +public: + using ColStringType = typename TypeTraits::FieldType; + using ColUInt8Type = typename TypeTraits::FieldType; + + ColumnWithTypeAndName escape = createConstColumn(1, static_cast('\\')); + + ColumnsWithTypeAndName lower_data11{toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa"))}; + ColumnsWithTypeAndName lower_data12{toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa"))}; + + ColumnsWithTypeAndName lower_data21{toVec("col0", std::vector(data_num, "AAAAAAAAAAAAAAAAA"))}; + ColumnsWithTypeAndName lower_data22{toVec("col1", std::vector(data_num, "AAAAAAAAAAAAAAAAA"))}; + + ColumnsWithTypeAndName lower_data31{toVec("col0", std::vector(data_num, "aAaAaAaAaAaAaAaAa"))}; + ColumnsWithTypeAndName lower_data32{toVec("col1", std::vector(data_num, "aAaAaAaAaAaAaAaAa"))}; + + ColumnsWithTypeAndName lower_data41{toVec("col0", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯"))}; + ColumnsWithTypeAndName lower_data42{toVec("col1", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯"))}; + + ColumnsWithTypeAndName lower_data51{toVec("col0", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯"))}; + ColumnsWithTypeAndName lower_data52{toVec("col1", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯"))}; + + ColumnsWithTypeAndName like_data1{ + toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + escape}; + ColumnsWithTypeAndName like_data2{ + toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + escape}; + ColumnsWithTypeAndName like_data3{ + toVec("col0", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + toVec("col1", std::vector(data_num, "aaaaaaaaaaaaaaaaa")), + escape}; + ColumnsWithTypeAndName like_data4{ + toVec("col0", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), + toVec("col1", std::vector(data_num, "嗯嗯嗯嗯嗯嗯嗯嗯嗯嗯")), + escape}; + ColumnsWithTypeAndName like_data5{ + toVec("col0", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), + toVec("col1", std::vector(data_num, "a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯a嗯")), + escape}; + + void SetUp(const benchmark::State &) override {} +}; + +BENCHMARK_DEFINE_F(IlikeBench, ilike) +(benchmark::State & state) +try +{ + FunctionIlike3Args function_ilike; + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_BIN); + function_ilike.setCollator(collator); + std::vector blocks{Block(data1), Block(data2), Block(data3), Block(data4), Block(data5)}; + for (auto & block : blocks) + block.insert({nullptr, std::make_shared>(), "res"}); + ColumnNumbers arguments{0, 1, 2}; + for (auto _ : state) + { + for (auto & block : blocks) + function_ilike.executeImpl(block, arguments, 3); + } +} +CATCH +BENCHMARK_REGISTER_F(IlikeBench, ilike)->Iterations(10); + +BENCHMARK_DEFINE_F(LikeBench, like) +(benchmark::State & state) +try +{ + FunctionLowerUTF8 function_lower; + FunctionLike function_like; + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8_BIN); + function_like.setCollator(collator); + std::vector lower_blocks{ + Block(lower_data11), + Block(lower_data21), + Block(lower_data31), + Block(lower_data41), + Block(lower_data51), + Block(lower_data12), + Block(lower_data22), + Block(lower_data32), + Block(lower_data42), + Block(lower_data52)}; + std::vector like_blocks{Block(like_data1), Block(like_data2), Block(like_data3), Block(like_data4), Block(like_data5)}; + + for (auto & block : lower_blocks) + block.insert({nullptr, std::make_shared(), "res"}); + for (auto & block : like_blocks) + block.insert({nullptr, std::make_shared>(), "res"}); + + ColumnNumbers lower_arguments{0, 1}; + ColumnNumbers like_arguments{0, 1, 2}; + for (auto _ : state) + { + for (auto & block : lower_blocks) + function_lower.executeImpl(block, lower_arguments, 1); + for (auto & block : like_blocks) + function_like.executeImpl(block, like_arguments, 3); + } +} +CATCH +BENCHMARK_REGISTER_F(LikeBench, like)->Iterations(10); + +<<<<<<< HEAD:dbms/src/Functions/tests/bench_function_ilike.cpp +======= +class CollationBench : public benchmark::Fixture +{ +public: + using ColStringType = typename TypeTraits::FieldType; + using ColUInt8Type = typename TypeTraits::FieldType; + + ColumnsWithTypeAndName data{ + toVec("col0", std::vector(1000000, "aaaaaaaaaaaaa")), + toVec("col1", std::vector(1000000, "aaaaaaaaaaaaa")), + toVec("result", std::vector{})}; + + ColumnsWithTypeAndName like_data{ + toVec("col0", std::vector(1000000, "qwdgefwabchfue")), + createConstColumn(1000000, "%abc%"), + createConstColumn(1000000, static_cast('\\')), + toVec("result", std::vector{})}; +}; + +class CollationLessBench : public CollationBench +{ +public: + void SetUp(const benchmark::State &) override {} +}; + +class CollationEqBench : public CollationBench +{ +public: + void SetUp(const benchmark::State &) override {} +}; + +class CollationLikeBench : public CollationBench +{ +public: + void SetUp(const benchmark::State &) override {} +}; + +#define BENCH_LESS_COLLATOR(collator) \ + BENCHMARK_DEFINE_F(CollationLessBench, collator) \ + (benchmark::State & state) \ + try \ + { \ + FunctionLess fl; \ + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ + fl.setCollator(collator); \ + Block block(data); \ + ColumnNumbers arguments{0, 1}; \ + for (auto _ : state) \ + { \ + fl.executeImpl(block, arguments, 2); \ + } \ + } \ + CATCH \ + BENCHMARK_REGISTER_F(CollationLessBench, collator)->Iterations(10); + + +#define BENCH_EQ_COLLATOR(collator) \ + BENCHMARK_DEFINE_F(CollationEqBench, collator) \ + (benchmark::State & state) \ + try \ + { \ + FunctionEquals fe; \ + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ + fe.setCollator(collator); \ + Block block(data); \ + ColumnNumbers arguments{0, 1}; \ + for (auto _ : state) \ + { \ + fe.executeImpl(block, arguments, 2); \ + } \ + } \ + CATCH \ + BENCHMARK_REGISTER_F(CollationEqBench, collator)->Iterations(10); + + +#define BENCH_LIKE_COLLATOR(collator) \ + BENCHMARK_DEFINE_F(CollationLikeBench, collator) \ + (benchmark::State & state) \ + try \ + { \ + FunctionLike3Args fl; \ + TiDB::TiDBCollatorPtr collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::collator); \ + fl.setCollator(collator); \ + Block block(like_data); \ + ColumnNumbers arguments{0, 1, 2}; \ + for (auto _ : state) \ + { \ + fl.executeImpl(block, arguments, 3); \ + } \ + } \ + CATCH \ + BENCHMARK_REGISTER_F(CollationLikeBench, collator)->Iterations(10); + + +BENCH_LESS_COLLATOR(UTF8MB4_BIN); +BENCH_LESS_COLLATOR(UTF8MB4_GENERAL_CI); +BENCH_LESS_COLLATOR(UTF8MB4_UNICODE_CI); +BENCH_LESS_COLLATOR(UTF8MB4_0900_AI_CI); +BENCH_LESS_COLLATOR(UTF8MB4_0900_BIN); +BENCH_LESS_COLLATOR(UTF8_BIN); +BENCH_LESS_COLLATOR(UTF8_GENERAL_CI); +BENCH_LESS_COLLATOR(UTF8_UNICODE_CI); +BENCH_LESS_COLLATOR(ASCII_BIN); +BENCH_LESS_COLLATOR(BINARY); +BENCH_LESS_COLLATOR(LATIN1_BIN); + +BENCH_EQ_COLLATOR(UTF8MB4_BIN); +BENCH_EQ_COLLATOR(UTF8MB4_GENERAL_CI); +BENCH_EQ_COLLATOR(UTF8MB4_UNICODE_CI); +BENCH_EQ_COLLATOR(UTF8MB4_0900_AI_CI); +BENCH_EQ_COLLATOR(UTF8MB4_0900_BIN); +BENCH_EQ_COLLATOR(UTF8_BIN); +BENCH_EQ_COLLATOR(UTF8_GENERAL_CI); +BENCH_EQ_COLLATOR(UTF8_UNICODE_CI); +BENCH_EQ_COLLATOR(ASCII_BIN); +BENCH_EQ_COLLATOR(BINARY); +BENCH_EQ_COLLATOR(LATIN1_BIN); + +BENCH_LIKE_COLLATOR(UTF8MB4_BIN); +BENCH_LIKE_COLLATOR(UTF8MB4_GENERAL_CI); +BENCH_LIKE_COLLATOR(UTF8MB4_UNICODE_CI); +BENCH_LIKE_COLLATOR(UTF8MB4_0900_AI_CI); +BENCH_LIKE_COLLATOR(UTF8MB4_0900_BIN); +BENCH_LIKE_COLLATOR(UTF8_BIN); +BENCH_LIKE_COLLATOR(UTF8_GENERAL_CI); +BENCH_LIKE_COLLATOR(UTF8_UNICODE_CI); +BENCH_LIKE_COLLATOR(ASCII_BIN); +BENCH_LIKE_COLLATOR(BINARY); +BENCH_LIKE_COLLATOR(LATIN1_BIN); + +class LengthBench : public benchmark::Fixture +{ +public: + using ColStringType = typename TypeTraits::FieldType; + + ColumnsWithTypeAndName data1{toVec("col", std::vector(data_num, ""))}; + ColumnsWithTypeAndName data2{toVec("col", std::vector(data_num, "aaaaaaaaaa"))}; + ColumnsWithTypeAndName data3{toVec("col", std::vector(data_num, "啊aaaaaaaa"))}; + + void SetUp(const benchmark::State &) override {} +}; + +BENCHMARK_DEFINE_F(LengthBench, bench) +(benchmark::State & state) +try +{ + FunctionLength function_length; + std::vector blocks{Block(data1), Block(data2), Block(data3)}; + for (auto & block : blocks) + block.insert({nullptr, std::make_shared>(), "res"}); + ColumnNumbers arguments{0}; + for (auto _ : state) + { + for (auto & block : blocks) + function_length.executeImpl(block, arguments, 1); + } +} +CATCH +BENCHMARK_REGISTER_F(LengthBench, bench)->Iterations(10); + +class ASCIIBench : public benchmark::Fixture +{ +public: + using ColStringType = typename TypeTraits::FieldType; + + ColumnsWithTypeAndName data1{toVec("col", std::vector(data_num, ""))}; + ColumnsWithTypeAndName data2{toVec("col", std::vector(data_num, "aaaaaaaaaa"))}; + ColumnsWithTypeAndName data3{toVec("col", std::vector(data_num, "啊aaaaaaaa"))}; + + void SetUp(const benchmark::State &) override {} +}; + +BENCHMARK_DEFINE_F(ASCIIBench, bench) +(benchmark::State & state) +try +{ + FunctionASCII function_ascii; + std::vector blocks{Block(data1), Block(data2), Block(data3)}; + for (auto & block : blocks) + block.insert({nullptr, std::make_shared>(), "res"}); + ColumnNumbers arguments{0}; + for (auto _ : state) + { + for (auto & block : blocks) + function_ascii.executeImpl(block, arguments, 1); + } +} +CATCH +BENCHMARK_REGISTER_F(ASCIIBench, bench)->Iterations(10); + +>>>>>>> b30c1f5090 (Improve the performance of `length` and `ascii` functions (#9345)):dbms/src/Functions/tests/bench_function_string.cpp +} // namespace tests +} // namespace DB diff --git a/dbms/src/Functions/tests/gtest_strings_ascii.cpp b/dbms/src/Functions/tests/gtest_strings_ascii.cpp index 2cb67e024c9..0c7a58559b0 100644 --- a/dbms/src/Functions/tests/gtest_strings_ascii.cpp +++ b/dbms/src/Functions/tests/gtest_strings_ascii.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #pragma GCC diagnostic push @@ -38,116 +37,49 @@ class StringASCII : public DB::tests::FunctionTest { }; -// test string and string TEST_F(StringASCII, strAndStrTest) { - const auto context = TiFlashTestEnv::getContext(); - - auto & factory = FunctionFactory::instance(); - - std::vector strs{"hello", "HELLO", "23333", "#%@#^", ""}; - - for (int i = 0; i < 2; i++) { - MutableColumnPtr csp; - csp = ColumnString::create(); - - for (const auto & str : strs) - { - csp->insert(Field(str.c_str(), str.size())); - } - - Block test_block; - ColumnWithTypeAndName ctn = ColumnWithTypeAndName(std::move(csp), std::make_shared(), "test_ascii"); - ColumnsWithTypeAndName ctns{ctn}; - test_block.insert(ctn); - ColumnNumbers cns{0}; - - // test ascii - auto bp = factory.tryGet("ascii", *context); - ASSERT_TRUE(bp != nullptr); - ASSERT_FALSE(bp->isVariadic()); - - auto func = bp->build(ctns); - test_block.insert({nullptr, func->getReturnType(), "res"}); - func->execute(test_block, cns, 1); - const IColumn * res = test_block.getByPosition(1).column.get(); - const ColumnInt64 * res_string = checkAndGetColumn(res); - - Field res_field; - std::vector results{104, 72, 50, 35, 0}; - for (size_t t = 0; t < results.size(); t++) - { - res_string->get(t, res_field); - Int64 res_val = res_field.get(); - EXPECT_EQ(results[t], res_val); - } + // test const + ASSERT_COLUMN_EQ(createConstColumn(0, 0), executeFunction("ascii", createConstColumn(0, ""))); + ASSERT_COLUMN_EQ( + createConstColumn(1, 38), + executeFunction("ascii", createConstColumn(1, "&ad"))); + ASSERT_COLUMN_EQ( + createConstColumn(5, 38), + executeFunction("ascii", createConstColumn(5, "&ad"))); } -} -// test NULL -TEST_F(StringASCII, nullTest) -{ - const auto context = TiFlashTestEnv::getContext(); - - auto & factory = FunctionFactory::instance(); - - std::vector strs{"a", "b", "c", "d", "e", "f"}; - std::vector results{0, 98, 0, 100, 101, 0}; - std::vector null_map{1, 0, 1, 0, 0, 1}; - auto input_str_col = ColumnString::create(); - for (const auto & str : strs) { - Field field(str.c_str(), str.size()); - input_str_col->insert(field); + // test vec + ASSERT_COLUMN_EQ(createColumn({}), executeFunction("ascii", createColumn({}))); + ASSERT_COLUMN_EQ( + createColumn({230, 104, 72, 50, 35, 0}), + executeFunction("ascii", createColumn({"我a", "hello", "HELLO", "23333", "#%@#^", ""}))); } - auto input_null_map = ColumnUInt8::create(strs.size(), 0); - ColumnUInt8::Container & input_vec_null_map = input_null_map->getData(); - for (size_t i = 0; i < strs.size(); i++) { - input_vec_null_map[i] = null_map[i]; + // test nullable const + ASSERT_COLUMN_EQ( + createConstColumn(0, {}), + executeFunction("ascii", createConstColumn>(0, "aaa"))); + ASSERT_COLUMN_EQ( + createConstColumn(1, {97}), + executeFunction("ascii", createConstColumn>(1, "aaa"))); + ASSERT_COLUMN_EQ( + createConstColumn(3, {97}), + executeFunction("ascii", createConstColumn>(3, "aaa"))); } - auto input_null_col = ColumnNullable::create(std::move(input_str_col), std::move(input_null_map)); - DataTypePtr string_type = std::make_shared(); - DataTypePtr nullable_string_type = makeNullable(string_type); - - auto col1 = ColumnWithTypeAndName(std::move(input_null_col), nullable_string_type, "ascii"); - ColumnsWithTypeAndName ctns{col1}; - - Block test_block; - test_block.insert(col1); - ColumnNumbers cns{0}; - - auto bp = factory.tryGet("ascii", *context); - ASSERT_TRUE(bp != nullptr); - ASSERT_FALSE(bp->isVariadic()); - auto func = bp->build(ctns); - test_block.insert({nullptr, func->getReturnType(), "res"}); - func->execute(test_block, cns, 1); - auto res_col = test_block.getByPosition(1).column; - - ColumnPtr result_null_map_column = static_cast(*res_col).getNullMapColumnPtr(); - MutableColumnPtr mutable_result_null_map_column = (*std::move(result_null_map_column)).mutate(); - NullMap & result_null_map = static_cast(*mutable_result_null_map_column).getData(); - const IColumn * res = test_block.getByPosition(1).column.get(); - const ColumnNullable * res_nullable_string = checkAndGetColumn(res); - const IColumn & res_string = res_nullable_string->getNestedColumn(); - - Field res_field; - - for (size_t i = 0; i < null_map.size(); i++) { - EXPECT_EQ(result_null_map[i], null_map[i]); - if (result_null_map[i] == 0) - { - res_string.get(i, res_field); - Int64 res_val = res_field.get(); - EXPECT_EQ(results[i], res_val); - } + // test nullable vec + std::vector null_map{0, 1, 0, 1, 0, 0, 1}; + ASSERT_COLUMN_EQ( + createNullableColumn({0, 0, 97, 0, 233, 233, 0}, null_map), + executeFunction( + "ascii", + createNullableColumn({"", "a", "abcd", "嗯", "饼干", "馒头", "???"}, null_map))); } } - } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Functions/tests/gtest_strings_length.cpp b/dbms/src/Functions/tests/gtest_strings_length.cpp index 4e0d95983ce..638cc783177 100644 --- a/dbms/src/Functions/tests/gtest_strings_length.cpp +++ b/dbms/src/Functions/tests/gtest_strings_length.cpp @@ -38,117 +38,53 @@ class StringLength : public DB::tests::FunctionTest { }; -// test string and string -TEST_F(StringLength, strAndStrTest) +TEST_F(StringLength, length) { - const auto context = TiFlashTestEnv::getContext(); - - auto & factory = FunctionFactory::instance(); - - std::vector strs{"hi~", "23333", "pingcap", "你好", "233哈哈", ""}; - std::vector results{3, 5, 7, 6, 9, 0}; - - for (int i = 0; i < 2; i++) { - MutableColumnPtr csp; - csp = ColumnString::create(); - - for (const auto & str : strs) - { - csp->insert(Field(str.c_str(), str.size())); - } - - Block test_block; - ColumnWithTypeAndName ctn = ColumnWithTypeAndName(std::move(csp), std::make_shared(), "test_ascii"); - ColumnsWithTypeAndName ctns{ctn}; - test_block.insert(ctn); - ColumnNumbers cns{0}; - - // test length - auto bp = factory.tryGet("length", *context); - ASSERT_TRUE(bp != nullptr); - ASSERT_FALSE(bp->isVariadic()); - - auto func = bp->build(ctns); - test_block.insert({nullptr, func->getReturnType(), "res"}); - func->execute(test_block, cns, 1); - const IColumn * res = test_block.getByPosition(1).column.get(); - const ColumnInt64 * res_string = checkAndGetColumn(res); - - Field res_field; - - for (size_t t = 0; t < results.size(); t++) - { - res_string->get(t, res_field); - Int64 res_val = res_field.get(); - EXPECT_EQ(results[t], res_val); - } + // test const + ASSERT_COLUMN_EQ(createConstColumn(0, 0), executeFunction("length", createConstColumn(0, ""))); + ASSERT_COLUMN_EQ( + createConstColumn(1, 3), + executeFunction("length", createConstColumn(1, "aaa"))); + ASSERT_COLUMN_EQ( + createConstColumn(3, 3), + executeFunction("length", createConstColumn(3, "aaa"))); } -} -// test NULL -TEST_F(StringLength, nullTest) -{ - const auto context = TiFlashTestEnv::getContext(); - - auto & factory = FunctionFactory::instance(); - - std::vector strs{"a", "abcd", "嗯", "饼干", "馒头", "???"}; - std::vector results{0, 4, 0, 6, 6, 0}; - std::vector null_map{1, 0, 1, 0, 0, 1}; - auto input_str_col = ColumnString::create(); - for (const auto & str : strs) { - Field field(str.c_str(), str.size()); - input_str_col->insert(field); + // test vec + ASSERT_COLUMN_EQ(createColumn({}), executeFunction("length", createColumn({}))); + + ASSERT_COLUMN_EQ( + createColumn({0, 3, 5, 7, 6, 9, 0, 9, 16, 0}), + executeFunction( + "length", + createColumn( + {"", "hi~", "23333", "pingcap", "你好", "233哈哈", "", "asdの的", "ヽ( ̄▽ ̄)و", ""}))); } - auto input_null_map = ColumnUInt8::create(strs.size(), 0); - ColumnUInt8::Container & input_vec_null_map = input_null_map->getData(); - for (size_t i = 0; i < strs.size(); i++) { - input_vec_null_map[i] = null_map[i]; + // test nullable const + ASSERT_COLUMN_EQ( + createConstColumn(0, {}), + executeFunction("length", createConstColumn>(0, "aaa"))); + ASSERT_COLUMN_EQ( + createConstColumn(1, {3}), + executeFunction("length", createConstColumn>(1, "aaa"))); + ASSERT_COLUMN_EQ( + createConstColumn(3, {3}), + executeFunction("length", createConstColumn>(3, "aaa"))); } - auto input_null_col = ColumnNullable::create(std::move(input_str_col), std::move(input_null_map)); - DataTypePtr string_type = std::make_shared(); - DataTypePtr nullable_string_type = makeNullable(string_type); - - auto col1 = ColumnWithTypeAndName(std::move(input_null_col), nullable_string_type, "length"); - ColumnsWithTypeAndName ctns{col1}; - - Block test_block; - test_block.insert(col1); - ColumnNumbers cns{0}; - - auto bp = factory.tryGet("length", *context); - ASSERT_TRUE(bp != nullptr); - ASSERT_FALSE(bp->isVariadic()); - auto func = bp->build(ctns); - test_block.insert({nullptr, func->getReturnType(), "res"}); - func->execute(test_block, cns, 1); - auto res_col = test_block.getByPosition(1).column; - - ColumnPtr result_null_map_column = static_cast(*res_col).getNullMapColumnPtr(); - MutableColumnPtr mutable_result_null_map_column = (*std::move(result_null_map_column)).mutate(); - NullMap & result_null_map = static_cast(*mutable_result_null_map_column).getData(); - const IColumn * res = test_block.getByPosition(1).column.get(); - const ColumnNullable * res_nullable_string = checkAndGetColumn(res); - const IColumn & res_string = res_nullable_string->getNestedColumn(); - - Field res_field; - - for (size_t i = 0; i < null_map.size(); i++) { - EXPECT_EQ(result_null_map[i], null_map[i]); - if (result_null_map[i] == 0) - { - res_string.get(i, res_field); - Int64 res_val = res_field.get(); - EXPECT_EQ(results[i], res_val); - } + // test nullable vec + std::vector null_map{1, 0, 1, 0, 0, 1}; + ASSERT_COLUMN_EQ( + createNullableColumn({0, 4, 0, 6, 6, 0}, null_map), + executeFunction( + "length", + createNullableColumn({"a", "abcd", "嗯", "饼干", "馒头", "???"}, null_map))); } } - } // namespace tests } // namespace DB diff --git a/dbms/src/Functions/tests/gtest_strings_position.cpp b/dbms/src/Functions/tests/gtest_strings_position.cpp index ac5b26fb84f..df5ac551f21 100644 --- a/dbms/src/Functions/tests/gtest_strings_position.cpp +++ b/dbms/src/Functions/tests/gtest_strings_position.cpp @@ -91,7 +91,7 @@ TEST_F(StringPosition, strAndStrTest) bp->build(ctns)->execute(test_block, cns, 2); const IColumn * res = test_block.getByPosition(2).column.get(); - const ColumnInt64 * res_string = checkAndGetColumn(res); + const auto * res_string = checkAndGetColumn(res); Field res_field; @@ -158,7 +158,7 @@ TEST_F(StringPosition, utf8StrAndStrTest) bp->build(ctns)->execute(test_block, cns, 2); const IColumn * res = test_block.getByPosition(2).column.get(); - const ColumnInt64 * res_string = checkAndGetColumn(res); + const auto * res_string = checkAndGetColumn(res); Field res_field; @@ -232,7 +232,7 @@ TEST_F(StringPosition, nullTest) MutableColumnPtr mutable_result_null_map_column = (*std::move(result_null_map_column)).mutate(); NullMap & result_null_map = static_cast(*mutable_result_null_map_column).getData(); const IColumn * res = test_block.getByPosition(2).column.get(); - const ColumnNullable * res_nullable_string = checkAndGetColumn(res); + const auto * res_nullable_string = checkAndGetColumn(res); const IColumn & res_string = res_nullable_string->getNestedColumn(); Field res_field; diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index f3dbcb38ac5..52bcbf884c4 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -26,7 +27,11 @@ #include #include #include +#include #include +#include + +#include namespace DB { @@ -37,6 +42,10 @@ extern const int UNKNOWN_TABLE; extern const int ILLFORMAT_RAFT_ROW; extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes +namespace FailPoints +{ +extern const char force_set_num_regions_for_table[]; +} // namespace FailPoints RegionTable::Table & RegionTable::getOrCreateTable(const KeyspaceID keyspace_id, const TableID table_id) { @@ -285,8 +294,8 @@ void RegionTable::removeRegion(const RegionID region_id, bool remove_data, const { tables.erase(ks_tb_id); } - LOG_INFO(log, "remove [region {}] in RegionTable done", region_id); } + LOG_INFO(log, "remove [region {}] in RegionTable done", region_id); // Sometime we don't need to remove data. e.g. remove region after region merge. if (remove_data) @@ -431,6 +440,31 @@ void RegionTable::handleInternalRegionsByTable(const KeyspaceID keyspace_id, con } } +std::vector RegionTable::getRegionIdsByTable(KeyspaceID keyspace_id, TableID table_id) const +{ + fiu_do_on(FailPoints::force_set_num_regions_for_table, { + if (auto v = FailPointHelper::getFailPointVal(FailPoints::force_set_num_regions_for_table); v) + { + auto num_regions = std::any_cast>(v.value()); + return num_regions; + } + }); + + std::lock_guard lock(mutex); + if (auto iter = tables.find(KeyspaceTableID{keyspace_id, table_id}); // + unlikely(iter != tables.end())) + { + std::vector ret_regions; + ret_regions.reserve(iter->second.regions.size()); + for (const auto & r : iter->second.regions) + { + ret_regions.emplace_back(r.first); + } + return ret_regions; + } + return {}; +} + std::vector> RegionTable::getRegionsByTable(const KeyspaceID keyspace_id, const TableID table_id) const { auto & kvstore = context->getTMTContext().getKVStore(); diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 95bcbd76aca..69ff456c8b0 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -170,6 +170,8 @@ class RegionTable : private boost::noncopyable RegionDataReadInfoList tryWriteBlockByRegionAndFlush(const RegionPtrWithBlock & region, bool try_persist); void handleInternalRegionsByTable(KeyspaceID keyspace_id, TableID table_id, std::function && callback) const; + + std::vector getRegionIdsByTable(KeyspaceID keyspace_id, TableID table_id) const; std::vector> getRegionsByTable(KeyspaceID keyspace_id, TableID table_id) const; /// Write the data of the given region into the table with the given table ID, fill the data list for outer to remove. diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.cpp b/dbms/src/TiDB/Schema/SchemaSyncService.cpp index 54cd4231f13..2fe1109188a 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.cpp +++ b/dbms/src/TiDB/Schema/SchemaSyncService.cpp @@ -41,15 +41,26 @@ SchemaSyncService::SchemaSyncService(DB::Context & context_) , log(Logger::get()) { // Add task for adding and removing keyspace sync schema tasks. - handle = background_pool.addTask( - [&, this] { - addKeyspaceGCTasks(); - removeKeyspaceGCTasks(); - - return false; - }, - false, - interval_seconds * 1000); + auto interval_ms = interval_seconds * 1000; + if (interval_ms == 0) + { + LOG_WARNING( + log, + "The background task of SchemaSyncService is disabled, please check the ddl_sync_interval_seconds " + "settings"); + } + else + { + handle = background_pool.addTask( + [&, this] { + addKeyspaceGCTasks(); + removeKeyspaceGCTasks(); + + return false; + }, + false, + interval_ms); + } } void SchemaSyncService::addKeyspaceGCTasks() @@ -142,7 +153,7 @@ void SchemaSyncService::removeKeyspaceGCTasks() LOG_IMPL(log, log_level, "remove sync schema task for keyspaces done, num_remove_tasks={}", num_remove_tasks); } -SchemaSyncService::~SchemaSyncService() +void SchemaSyncService::shutdown() { background_pool.removeTask(handle); for (auto const & iter : ks_handle_map) @@ -153,6 +164,11 @@ SchemaSyncService::~SchemaSyncService() LOG_INFO(log, "SchemaSyncService stopped"); } +SchemaSyncService::~SchemaSyncService() +{ + shutdown(); +} + bool SchemaSyncService::syncSchemas(KeyspaceID keyspace_id) { return context.getTMTContext().getSchemaSyncer()->syncSchemas(context, keyspace_id); @@ -187,6 +203,11 @@ void SchemaSyncService::updateLastGcSafepoint(KeyspaceID keyspace_id, Timestamp } bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) +{ + return gcImpl(gc_safepoint, keyspace_id, /*ignore_remain_regions*/ false); +} + +bool SchemaSyncService::gcImpl(Timestamp gc_safepoint, KeyspaceID keyspace_id, bool ignore_remain_regions) { const std::optional last_gc_safepoint = lastGcSafePoint(keyspace_id); // for new deploy cluster, there is an interval that gc_safepoint return 0, skip it @@ -242,6 +263,7 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } } + auto & tmt_context = context.getTMTContext(); // Physically drop tables bool succeeded = true; for (auto & storage_ptr : storages_to_gc) @@ -267,6 +289,38 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) *database_id, table_info.id); }(); + + auto & region_table = tmt_context.getRegionTable(); + if (auto remain_regions = region_table.getRegionIdsByTable(keyspace_id, table_info.id); // + !remain_regions.empty()) + { + if (likely(!ignore_remain_regions)) + { + LOG_WARNING( + keyspace_log, + "Physically drop table is skip, regions are not totally removed from TiFlash, remain_region_ids={}" + " table_tombstone={} safepoint={} {}", + remain_regions, + storage->getTombstone(), + gc_safepoint, + canonical_name); + succeeded = false; // dropping this table is skipped, do not succee the `last_gc_safepoint` + continue; + } + else + { + LOG_WARNING( + keyspace_log, + "Physically drop table is executed while regions are not totally removed from TiFlash," + " remain_region_ids={} ignore_remain_regions={} table_tombstone={} safepoint={} {} ", + remain_regions, + ignore_remain_regions, + storage->getTombstone(), + gc_safepoint, + canonical_name); + } + } + LOG_INFO( keyspace_log, "Physically drop table begin, table_tombstone={} safepoint={} {}", @@ -288,7 +342,7 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } catch (DB::Exception & e) { - succeeded = false; + succeeded = false; // dropping this table is skipped, do not succee the `last_gc_safepoint` String err_msg; // Maybe a read lock of a table is held for a long time, just ignore it this round. if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) @@ -338,7 +392,7 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } catch (DB::Exception & e) { - succeeded = false; + succeeded = false; // dropping this database is skipped, do not succee the `last_gc_safepoint` String err_msg; if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) err_msg = "locking attempt has timed out!"; // ignore verbose stack for this error @@ -348,6 +402,8 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } } + // TODO: Optimize it after `BackgroundProcessingPool` can the task return how many seconds to sleep + // before next round. if (succeeded) { updateLastGcSafepoint(keyspace_id, gc_safepoint); @@ -357,6 +413,11 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) num_tables_removed, num_databases_removed, gc_safepoint); + // This round of GC could run for a long time. Run immediately to check whether + // the latest gc_safepoint has been updated in PD. + // - gc_safepoint is not updated, it will be skipped because gc_safepoint == last_gc_safepoint + // - gc_safepoint is updated, run again immediately to cleanup other dropped data + return true; } else { @@ -366,9 +427,10 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) "Schema GC meet error, will try again later, last_safepoint={} safepoint={}", last_gc_safepoint_str, gc_safepoint); + // Return false to let it run again after `ddl_sync_interval_seconds` even if the gc_safepoint + // on PD is not updated. + return false; } - - return true; } } // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.h b/dbms/src/TiDB/Schema/SchemaSyncService.h index 77257ace6e0..20e2e67f28f 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.h +++ b/dbms/src/TiDB/Schema/SchemaSyncService.h @@ -35,6 +35,11 @@ using ASTs = std::vector; using DBGInvokerPrinter = std::function; extern void dbgFuncGcSchemas(Context &, const ASTs &, DBGInvokerPrinter); +namespace tests +{ +class SchemaSyncTest; +} + class SchemaSyncService : public std::enable_shared_from_this , private boost::noncopyable @@ -43,17 +48,22 @@ class SchemaSyncService explicit SchemaSyncService(Context & context_); ~SchemaSyncService(); + friend class tests::SchemaSyncTest; + bool gc(Timestamp gc_safepoint, KeyspaceID keyspace_id); + + void shutdown(); + private: bool syncSchemas(KeyspaceID keyspace_id); void removeCurrentVersion(KeyspaceID keyspace_id); - bool gc(Timestamp gc_safepoint, KeyspaceID keyspace_id); void addKeyspaceGCTasks(); void removeKeyspaceGCTasks(); std::optional lastGcSafePoint(KeyspaceID keyspace_id) const; void updateLastGcSafepoint(KeyspaceID keyspace_id, Timestamp gc_safepoint); + bool gcImpl(Timestamp gc_safepoint, KeyspaceID keyspace_id, bool ignore_remain_regions); private: Context & context; diff --git a/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp b/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp index c0613e82696..8c5cf561b55 100644 --- a/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp +++ b/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp @@ -39,6 +39,7 @@ namespace FailPoints extern const char exception_before_rename_table_old_meta_removed[]; extern const char force_schema_sync_too_old_schema[]; extern const char force_context_path[]; +extern const char force_set_num_regions_for_table[]; } // namespace FailPoints namespace tests { @@ -167,6 +168,11 @@ class SchemaSyncTest : public ::testing::Test drop_interpreter.execute(); } + static std::optional lastGcSafePoint(const SchemaSyncServicePtr & sync_service, KeyspaceID keyspace_id) + { + return sync_service->lastGcSafePoint(keyspace_id); + } + private: static void recreateMetadataPath() { @@ -187,7 +193,7 @@ try // Note that if we want to add new fields here, please firstly check if it is present. // Otherwise it will break when doing upgrading test. SchemaDiff diff; - std::string data = "{\"version\":40,\"type\":31,\"schema_id\":69,\"table_id\":71,\"old_table_id\":0,\"old_schema_id\":0,\"affected_options\":null}"; + std::string data = R"({"version":40,"type":31,"schema_id":69,"table_id":71,"old_table_id":0,"old_schema_id":0,"affected_options":null})"; ASSERT_NO_THROW(diff.deserialize(data)); } CATCH @@ -230,6 +236,113 @@ try } CATCH +TEST_F(SchemaSyncTest, PhysicalDropTable) +try +{ + auto pd_client = global_ctx.getTMTContext().getPDClient(); + + const String db_name = "mock_db"; + MockTiDB::instance().newDataBase(db_name); + + auto cols = ColumnsDescription({ + {"col1", typeFromString("String")}, + {"col2", typeFromString("Int64")}, + }); + // table_name, cols, pk_name + std::vector> tables{ + {"t1", cols, ""}, + {"t2", cols, ""}, + }; + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + + refreshSchema(); + + mustGetSyncedTableByName(db_name, "t1"); + mustGetSyncedTableByName(db_name, "t2"); + + MockTiDB::instance().dropTable(global_ctx, db_name, "t1", true); + + refreshSchema(); + + // Create a temporary context with ddl sync task disabled + auto sync_service = std::make_shared(global_ctx); + sync_service->shutdown(); // shutdown the background tasks + + // run gc with safepoint == 0, will be skip + ASSERT_FALSE(sync_service->gc(0, NullspaceID)); + ASSERT_TRUE(sync_service->gc(10000000, NullspaceID)); + // run gc with the same safepoint, will be skip + ASSERT_FALSE(sync_service->gc(10000000, NullspaceID)); + // run gc for another keyspace with same safepoint, will be executed + ASSERT_TRUE(sync_service->gc(10000000, 1024)); + // run gc with changed safepoint + ASSERT_TRUE(sync_service->gc(20000000, 1024)); + // run gc with the same safepoint + ASSERT_FALSE(sync_service->gc(20000000, 1024)); +} +CATCH + +TEST_F(SchemaSyncTest, PhysicalDropTableMeetsUnRemovedRegions) +try +{ + auto pd_client = global_ctx.getTMTContext().getPDClient(); + + const String db_name = "mock_db"; + MockTiDB::instance().newDataBase(db_name); + + auto cols = ColumnsDescription({ + {"col1", typeFromString("String")}, + {"col2", typeFromString("Int64")}, + }); + // table_name, cols, pk_name + std::vector> tables{ + {"t1", cols, ""}, + }; + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + + refreshSchema(); + + mustGetSyncedTableByName(db_name, "t1"); + + MockTiDB::instance().dropTable(global_ctx, db_name, "t1", true); + + refreshSchema(); + + // prevent the storage instance from being physically removed + FailPointHelper::enableFailPoint( + FailPoints::force_set_num_regions_for_table, + std::vector{1001, 1002, 1003}); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::force_set_num_regions_for_table); }); + + auto sync_service = std::make_shared(global_ctx); + sync_service->shutdown(); // shutdown the background tasks + + { + // ensure gc_safe_point cache is empty + auto last_gc_safe_point = lastGcSafePoint(sync_service, NullspaceID); + ASSERT_FALSE(last_gc_safe_point.has_value()); + } + + // Run GC, but the table is not physically dropped because `force_set_num_regions_for_table` + ASSERT_FALSE(sync_service->gc(std::numeric_limits::max(), NullspaceID)); + { + // gc_safe_point cache is not updated + auto last_gc_safe_point = lastGcSafePoint(sync_service, NullspaceID); + ASSERT_FALSE(last_gc_safe_point.has_value()); + } + + // ensure the table is not physically dropped + size_t num_remain_tables = 0; + for (auto table_id : table_ids) + { + auto storage = global_ctx.getTMTContext().getStorages().get(NullspaceID, table_id); + ASSERT_TRUE(storage->isTombstone()); + ++num_remain_tables; + } + ASSERT_EQ(num_remain_tables, 1); +} +CATCH + TEST_F(SchemaSyncTest, RenamePartitionTable) try { diff --git a/libs/libcommon/include/common/StringRef.h b/libs/libcommon/include/common/StringRef.h index 89328c6b3db..5018d3253b7 100644 --- a/libs/libcommon/include/common/StringRef.h +++ b/libs/libcommon/include/common/StringRef.h @@ -23,7 +23,6 @@ #include #include #include -#include // for std::logic_error #include #include @@ -172,8 +171,8 @@ inline size_t hashLessThan8(const char * data, size_t size) { if (size > 8) { - UInt64 a = unalignedLoad(data); - UInt64 b = unalignedLoad(data + size - 8); + auto a = unalignedLoad(data); + auto b = unalignedLoad(data + size - 8); return hashLen16(a, rotateByAtLeast1(b + size, size)) ^ b; } @@ -200,13 +199,13 @@ struct CRC32Hash do { - UInt64 word = unalignedLoad(pos); + auto word = unalignedLoad(pos); res = _mm_crc32_u64(res, word); pos += 8; } while (pos + 8 < end); - UInt64 word = unalignedLoad(end - 8); /// I'm not sure if this is normal. + auto word = unalignedLoad(end - 8); /// I'm not sure if this is normal. res = _mm_crc32_u64(res, word); return res; diff --git a/tests/fullstack-test/expr/cast_as_decimal.test b/tests/fullstack-test/expr/cast_as_decimal.test index aa85c592ec4..a375433b843 100644 --- a/tests/fullstack-test/expr/cast_as_decimal.test +++ b/tests/fullstack-test/expr/cast_as_decimal.test @@ -35,3 +35,28 @@ mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; +------------------------------------+ | 20221010101010.123 | +------------------------------------+ + +mysql> drop table if exists test.t2; +mysql> create table test.t2(d decimal(11, 4)); +mysql> alter table test.t2 set tiflash replica 1; +mysql> insert into test.t2 values(-0.741); +mysql> alter table test.t2 set tiflash replica 1; +func> wait_table test t2 +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; SELECT cast(d as decimal) from test.t2; ++--------------------+ +| cast(d as decimal) | ++--------------------+ +| -1 | ++--------------------+ + +mysql> drop table if exists test.t2; +mysql> create table test.t2 (c1 int not null, c2 int not null, primary key(c1) CLUSTERED); +mysql> alter table test.t2 set tiflash replica 1; +mysql> insert into test.t2 (c1,c2) values (1486109909, -1113200806); +func> wait_table test t2 +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c2, c1, cast( (c2 / cast(c1 as signed)) as decimal) as c2 from test.t2; ++-------------+------------+------+ +| c2 | c1 | c2 | ++-------------+------------+------+ +| -1113200806 | 1486109909 | -1 | ++-------------+------------+------+ diff --git a/tests/fullstack-test2/ddl/alter_exchange_partition.test b/tests/fullstack-test2/ddl/alter_exchange_partition.test index d393b1fc69f..7982a6ffbf2 100644 --- a/tests/fullstack-test2/ddl/alter_exchange_partition.test +++ b/tests/fullstack-test2/ddl/alter_exchange_partition.test @@ -303,7 +303,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 # ensure the swap out table is not mark as tombstone >> DBGInvoke __enable_schema_sync_service('true') ->> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __gc_schemas(18446744073709551615, 'true') >> DBGInvoke __enable_schema_sync_service('false') mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; +-----+-------+-------+ @@ -348,7 +348,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 +-----+-------+-------+ # ensure the swap out table is not mark as tombstone >> DBGInvoke __enable_schema_sync_service('true') ->> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __gc_schemas(18446744073709551615, 'true') >> DBGInvoke __enable_schema_sync_service('false') mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; +-----+-------+-------+ diff --git a/tests/fullstack-test2/ddl/flashback_database.test b/tests/fullstack-test2/ddl/flashback_database.test index 733e40b0d64..80f81f02299 100644 --- a/tests/fullstack-test2/ddl/flashback_database.test +++ b/tests/fullstack-test2/ddl/flashback_database.test @@ -55,7 +55,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from d1_new.t # ensure the flashbacked table and database is not mark as tombstone >> DBGInvoke __enable_schema_sync_service('true') ->> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __gc_schemas(18446744073709551615, 'true') mysql> set session tidb_isolation_read_engines='tiflash'; select * from d1_new.t3 order by a; +------+------+