From b54dece5da237bff4ffa329093bc7e4590d717b7 Mon Sep 17 00:00:00 2001 From: ywqzzy <592838129@qq.com> Date: Mon, 14 Mar 2022 15:19:46 +0800 Subject: [PATCH] update. Signed-off-by: ywqzzy <592838129@qq.com> update. Signed-off-by: ywqzzy <592838129@qq.com> update. Signed-off-by: ywqzzy <592838129@qq.com> Split write and ingest throughput (#4245) close pingcap/tiflash#4243 Signed-off-by: ywqzzy <592838129@qq.com> fix the problem that expired data was not recycled timely due to slow gc speed (#4224) close pingcap/tiflash#4146 Signed-off-by: ywqzzy <592838129@qq.com> PageStorage: Fix some bugs (#4212) ref pingcap/tiflash#3594 Signed-off-by: ywqzzy <592838129@qq.com> Make test of CreateTables using affected opts (#4239) close pingcap/tiflash#4235 Signed-off-by: ywqzzy <592838129@qq.com> BlobStore: remove the old_ids, not need recycle the blob id. (#4247) ref pingcap/tiflash#3594 Signed-off-by: ywqzzy <592838129@qq.com> support `PartitionTableScan` in TiFlash (#3876) close pingcap/tiflash#3873 Signed-off-by: ywqzzy <592838129@qq.com> update. Signed-off-by: ywqzzy <592838129@qq.com> update. Signed-off-by: ywqzzy <592838129@qq.com> support last_day and dayofmonth pushdown to tiflash (#4183) close pingcap/tiflash#4149 Signed-off-by: ywqzzy <592838129@qq.com> --- contrib/kvproto | 2 +- contrib/tipb | 2 +- dbms/CMakeLists.txt | 1 + dbms/src/Common/Exception.h | 6 + dbms/src/Common/MyTime.cpp | 10 +- dbms/src/Common/MyTime.h | 17 + dbms/src/Common/TiFlashMetrics.h | 3 +- dbms/src/Core/Types.h | 1 - dbms/src/DataTypes/DataTypeNullable.cpp | 11 +- dbms/src/DataTypes/DataTypeNullable.h | 23 +- dbms/src/Debug/MockTiDB.cpp | 37 +- dbms/src/Debug/dbgFuncCoprocessor.cpp | 139 ++-- dbms/src/Flash/BatchCoprocessorHandler.cpp | 25 +- dbms/src/Flash/BatchCoprocessorHandler.h | 4 +- dbms/src/Flash/Coprocessor/DAGContext.cpp | 10 + dbms/src/Flash/Coprocessor/DAGContext.h | 30 +- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 2 +- .../Coprocessor/DAGExpressionAnalyzer.cpp | 20 +- .../DAGExpressionAnalyzerHelper.cpp | 4 +- dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp | 7 +- dbms/src/Flash/Coprocessor/DAGQueryBlock.h | 1 + .../Coprocessor/DAGQueryBlockInterpreter.cpp | 135 ++-- .../Coprocessor/DAGQueryBlockInterpreter.h | 9 +- .../Coprocessor/DAGStorageInterpreter.cpp | 661 ++++++++++-------- .../Flash/Coprocessor/DAGStorageInterpreter.h | 49 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 4 +- dbms/src/Flash/Coprocessor/RemoteRequest.cpp | 92 +++ dbms/src/Flash/Coprocessor/RemoteRequest.h | 34 + .../Flash/Coprocessor/TablesRegionsInfo.cpp | 97 +++ .../src/Flash/Coprocessor/TablesRegionsInfo.h | 76 ++ dbms/src/Flash/Coprocessor/TiDBTableScan.cpp | 62 ++ dbms/src/Flash/Coprocessor/TiDBTableScan.h | 55 ++ .../Coprocessor/collectOutputFieldTypes.cpp | 7 +- dbms/src/Flash/CoprocessorHandler.cpp | 17 +- dbms/src/Flash/CoprocessorHandler.h | 6 +- dbms/src/Flash/Mpp/MPPHandler.cpp | 13 +- dbms/src/Flash/Mpp/MPPTask.cpp | 55 +- dbms/src/Flash/Statistics/TableScanImpl.h | 2 +- .../Flash/Statistics/traverseExecutors.cpp | 1 + dbms/src/Functions/FunctionsCoding.h | 15 +- dbms/src/Functions/FunctionsComparison.h | 3 +- dbms/src/Functions/FunctionsConversion.h | 8 +- dbms/src/Functions/FunctionsDateTime.cpp | 1 + dbms/src/Functions/FunctionsDateTime.h | 124 +++- dbms/src/Functions/FunctionsLogical.h | 2 +- dbms/src/Functions/FunctionsString.cpp | 33 +- .../tests/gtest_datetime_last_day.cpp | 103 +++ .../Interpreters/InterpreterSelectQuery.cpp | 48 +- dbms/src/Interpreters/Settings.h | 4 +- .../Storages/DeltaMerge/File/DMFileWriter.cpp | 2 +- dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp | 31 +- .../tests/gtest_dm_storage_delta_merge.cpp | 27 +- dbms/src/Storages/MutableSupport.cpp | 3 +- dbms/src/Storages/Page/V3/BlobStore.cpp | 31 +- dbms/src/Storages/Page/V3/BlobStore.h | 3 - dbms/src/Storages/Page/V3/PageDirectory.cpp | 46 +- dbms/src/Storages/Page/V3/PageDirectory.h | 4 +- .../Storages/Page/V3/PageDirectoryFactory.cpp | 30 +- dbms/src/Storages/Page/V3/PageEntriesEdit.h | 45 +- dbms/src/Storages/Page/V3/WAL/WALReader.cpp | 14 +- dbms/src/Storages/Page/V3/WAL/serialize.cpp | 2 +- dbms/src/Storages/Page/V3/WALStore.cpp | 29 +- .../Storages/Page/V3/tests/entries_helper.h | 2 +- .../Page/V3/tests/gtest_blob_store.cpp | 34 +- .../Page/V3/tests/gtest_page_directory.cpp | 156 ++++- .../Page/V3/tests/gtest_page_storage.cpp | 41 ++ .../Page/V3/tests/gtest_wal_store.cpp | 57 +- dbms/src/Storages/RegionQueryInfo.h | 10 +- dbms/src/Storages/Transaction/LearnerRead.cpp | 26 +- dbms/src/Storages/Transaction/LearnerRead.h | 2 +- metrics/grafana/tiflash_summary.json | 90 ++- .../query/mpp/partition_table.test | 103 +++ tests/fullstack-test/expr/day_of_month.test | 27 + tests/fullstack-test/expr/last_day.test | 147 ++++ .../mpp/dynamic_partition_prune.test | 62 ++ 75 files changed, 2326 insertions(+), 769 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/RemoteRequest.cpp create mode 100644 dbms/src/Flash/Coprocessor/RemoteRequest.h create mode 100644 dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp create mode 100644 dbms/src/Flash/Coprocessor/TablesRegionsInfo.h create mode 100644 dbms/src/Flash/Coprocessor/TiDBTableScan.cpp create mode 100644 dbms/src/Flash/Coprocessor/TiDBTableScan.h create mode 100644 dbms/src/Functions/tests/gtest_datetime_last_day.cpp create mode 100644 tests/delta-merge-test/query/mpp/partition_table.test create mode 100644 tests/fullstack-test/expr/day_of_month.test create mode 100644 tests/fullstack-test/expr/last_day.test create mode 100644 tests/fullstack-test/mpp/dynamic_partition_prune.test diff --git a/contrib/kvproto b/contrib/kvproto index 714e05ea3b1..d229fcc888c 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 714e05ea3b18f9f89e88ae88db693bd5e0b383a2 +Subproject commit d229fcc888c88506e1a81be0bc19df56623b99da diff --git a/contrib/tipb b/contrib/tipb index 0e3817b1f55..d12dec7a760 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit 0e3817b1f556337705053dac55606d04030bf1a0 +Subproject commit d12dec7a76095ac7c1db102948e7bf9ebaa970c1 diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index cadcd438b09..07dc8a269e9 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -177,6 +177,7 @@ target_link_libraries (clickhouse_common_io cpptoml ) target_include_directories (clickhouse_common_io BEFORE PRIVATE ${kvClient_SOURCE_DIR}/include) +target_include_directories (clickhouse_common_io BEFORE PUBLIC ${kvproto_SOURCE_DIR} ${tipb_SOURCE_DIR} ${Protobuf_INCLUDE_DIR} ${gRPC_INCLUDE_DIRS}) target_link_libraries (dbms clickhouse_parsers diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index 0354253b3b0..49b29075e42 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -36,6 +36,12 @@ class Exception : public Poco::Exception : Poco::Exception(exc.displayText()) {} + Exception(const Poco::Exception & exc, const std::string & arg) + : Poco::Exception(exc.displayText()) + { + extendedMessage(arg); + } + const char * name() const throw() override { return "DB::Exception"; } const char * className() const throw() override { return "DB::Exception"; } DB::Exception * clone() const override { return new DB::Exception(*this); } diff --git a/dbms/src/Common/MyTime.cpp b/dbms/src/Common/MyTime.cpp index d4a97ea06c0..53d4d5800d9 100644 --- a/dbms/src/Common/MyTime.cpp +++ b/dbms/src/Common/MyTime.cpp @@ -523,15 +523,7 @@ bool checkTimeValid(Int32 year, Int32 month, Int32 day, Int32 hour, Int32 minute { return false; } - static int days_of_month_table[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - if (month != 2) - return day <= days_of_month_table[month]; - bool is_leap_year = false; - if ((year & 0b0011) == 0) - { - is_leap_year = year % 100 != 0 || year % 400 == 0; - } - return day <= (is_leap_year ? 29 : 28); + return day <= getLastDay(year, month); } std::pair parseMyDateTimeAndJudgeIsDate(const String & str, int8_t fsp, bool needCheckTimeValid) diff --git a/dbms/src/Common/MyTime.h b/dbms/src/Common/MyTime.h index 90258e25b1b..da6be24fe45 100644 --- a/dbms/src/Common/MyTime.h +++ b/dbms/src/Common/MyTime.h @@ -192,4 +192,21 @@ bool isValidSeperator(char c, int previous_parts); // Note that this function will not check if the input is logically a valid datetime value. bool toCoreTimeChecked(const UInt64 & year, const UInt64 & month, const UInt64 & day, const UInt64 & hour, const UInt64 & minute, const UInt64 & second, const UInt64 & microsecond, MyDateTime & result); +inline bool isLeapYear(UInt16 year) +{ + return ((year % 4 == 0) && (year % 100 != 0)) || (year % 400 == 0); +} + +// Get last day of a month. Return 0 if month if invalid. +inline UInt8 getLastDay(UInt16 year, UInt8 month) +{ + static constexpr UInt8 days_of_month_table[] = {0, 31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + UInt8 last_day = 0; + if (month > 0 && month <= 12) + last_day = days_of_month_table[month]; + if (month == 2 && isLeapYear(year)) + last_day = 29; + return last_day; +} + } // namespace DB diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 8a036ccf12b..fb4def3c3ef 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -40,7 +40,8 @@ namespace DB M(tiflash_coprocessor_executor_count, "Total number of each executor", Counter, F(type_ts, {"type", "table_scan"}), \ F(type_sel, {"type", "selection"}), F(type_agg, {"type", "aggregation"}), F(type_topn, {"type", "top_n"}), \ F(type_limit, {"type", "limit"}), F(type_join, {"type", "join"}), F(type_exchange_sender, {"type", "exchange_sender"}), \ - F(type_exchange_receiver, {"type", "exchange_receiver"}), F(type_projection, {"type", "projection"})) \ + F(type_exchange_receiver, {"type", "exchange_receiver"}), F(type_projection, {"type", "projection"}), \ + F(type_partition_ts, {"type", "partition_table_scan"})) \ M(tiflash_coprocessor_request_duration_seconds, "Bucketed histogram of request duration", Histogram, \ F(type_batch, {{"type", "batch"}}, ExpBuckets{0.0005, 2, 30}), F(type_cop, {{"type", "cop"}}, ExpBuckets{0.0005, 2, 30}), \ F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{0.0005, 2, 30}), \ diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index 30b8e42fde2..77a20ed94a5 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -133,7 +133,6 @@ enum class TypeIndex Tuple, Set, Interval, - Nullable, Function, AggregateFunction, LowCardinality, diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index 254bcc61d81..7696cf5052f 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -58,7 +58,7 @@ void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( /// First serialize null map. path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + if (auto * stream = getter(path)) DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); /// Then serialize contents of arrays. @@ -78,7 +78,7 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( ColumnNullable & col = static_cast(column); path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + if (auto * stream = getter(path)) DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); path.back() = Substream::NullableElements; @@ -99,7 +99,7 @@ void DataTypeNullable::serializeWidenBinaryBulkWithMultipleStreams( /// First serialize null map. path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + if (auto * stream = getter(path)) DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); /// Then serialize contents of arrays. @@ -119,7 +119,7 @@ void DataTypeNullable::deserializeWidenBinaryBulkWithMultipleStreams( ColumnNullable & col = static_cast(column); path.push_back(Substream::NullMap); - if (auto stream = getter(path)) + if (auto * stream = getter(path)) DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); path.back() = Substream::NullableElements; @@ -172,7 +172,7 @@ void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) co { safeDeserialize( column, - [&istr] { bool is_null = 0; readBinary(is_null, istr); return is_null; }, + [&istr] { bool is_null = false; readBinary(is_null, istr); return is_null; }, [this, &istr](IColumn & nested) { nested_data_type->deserializeBinary(nested, istr); }); } @@ -337,7 +337,6 @@ bool DataTypeNullable::equals(const IDataType & rhs) const return rhs.isNullable() && nested_data_type->equals(*static_cast(rhs).nested_data_type); } - static DataTypePtr create(const ASTPtr & arguments) { if (!arguments || arguments->children.size() != 1) diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index f547795d4d0..124d8c29fbc 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -12,12 +12,11 @@ class DataTypeNullable final : public IDataType public: static constexpr bool is_parametric = true; - DataTypeNullable(const DataTypePtr & nested_data_type_); + explicit DataTypeNullable(const DataTypePtr & nested_data_type_); std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } - TypeIndex getTypeId() const override { return TypeIndex::Nullable; } - + TypeIndex getTypeId() const override { return nested_data_type->getTypeId(); } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; void serializeBinaryBulkWithMultipleStreams( @@ -70,7 +69,7 @@ class DataTypeNullable final : public IDataType * Now we support only first. * In CSV, non-NULL string value, starting with \N characters, must be placed in quotes, to avoid ambiguity. */ - void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, char delimiter) const override; void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettingsJSON &) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override; @@ -92,11 +91,27 @@ class DataTypeNullable final : public IDataType bool canBeComparedWithCollation() const override { return nested_data_type->canBeComparedWithCollation(); } bool canBeUsedAsVersion() const override { return false; } bool isSummable() const override { return nested_data_type->isSummable(); } + bool canBeUsedInBitOperations() const override { return nested_data_type->canBeUsedInBitOperations(); }; bool canBeUsedInBooleanContext() const override { return nested_data_type->canBeUsedInBooleanContext(); } + bool isNumber() const override { return nested_data_type->isNumber(); } + bool isInteger() const override { return nested_data_type->isInteger(); } + bool isUnsignedInteger() const override { return nested_data_type->isUnsignedInteger(); } + bool isFloatingPoint() const override { return nested_data_type->isFloatingPoint(); } + bool isDateOrDateTime() const override { return nested_data_type->isDateOrDateTime(); } // used in delta merge + bool isMyDateOrMyDateTime() const override { return nested_data_type->isMyDateOrMyDateTime(); } + bool isMyTime() const override { return nested_data_type->isMyTime(); } + bool isDecimal() const override { return nested_data_type->isDecimal(); } + bool isValueRepresentedByNumber() const override { return nested_data_type->isValueRepresentedByNumber(); } + bool isValueRepresentedByInteger() const override { return nested_data_type->isValueRepresentedByInteger(); } + bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return nested_data_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion(); } + bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return nested_data_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion(); } + bool isString() const override { return nested_data_type->isString(); } bool haveMaximumSizeOfValue() const override { return nested_data_type->haveMaximumSizeOfValue(); } size_t getMaximumSizeOfValueInMemory() const override { return 1 + nested_data_type->getMaximumSizeOfValueInMemory(); } bool isNullable() const override { return true; } size_t getSizeOfValueInMemory() const override; + bool isCategorial() const override { return nested_data_type->isCategorial(); } + bool isEnum() const override { return nested_data_type->isEnum(); } bool onlyNull() const override; const DataTypePtr & getNestedType() const { return nested_data_type; } diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index 90626ae69dd..09bb114c86c 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -263,10 +263,45 @@ int MockTiDB::newTables( Timestamp tso, const String & engine_type) { + std::lock_guard lock(tables_mutex); + if (databases.find(database_name) == databases.end()) + { + throw Exception("MockTiDB not found db: " + database_name, ErrorCodes::LOGICAL_ERROR); + } + + version++; + SchemaDiff diff; + diff.type = SchemaActionType::CreateTables; for (const auto & [table_name, columns, handle_pk_name] : tables) { - newTable(database_name, table_name, columns, tso, handle_pk_name, engine_type); + String qualified_name = database_name + "." + table_name; + if (tables_by_name.find(qualified_name) != tables_by_name.end()) + { + throw Exception("Mock TiDB table " + qualified_name + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS); + } + + auto table_info = *parseColumns(table_name, columns, handle_pk_name, engine_type); + 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)); + tables_by_id.emplace(table->table_info.id, table); + tables_by_name.emplace(qualified_name, table); + + AffectedOption opt; + opt.schema_id = table->database_id; + opt.table_id = table->id(); + opt.old_schema_id = table->database_id; + opt.old_table_id = table->id(); + diff.affected_opts.push_back(std::move(opt)); } + + if (diff.affected_opts.empty()) + throw Exception("MockTiDB CreateTables should have at lease 1 table", ErrorCodes::LOGICAL_ERROR); + + diff.schema_id = diff.affected_opts[0].schema_id; + diff.version = version; + version_diff[version] = diff; return 0; } diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index 1932d4213b4..8e49895aa02 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -261,6 +261,18 @@ DAGProperties getDAGProperties(const String & prop_string) return ret; } +void setTipbRegionInfo(coprocessor::RegionInfo * tipb_region_info, const std::pair & region, TableID table_id) +{ + tipb_region_info->set_region_id(region.first); + auto * meta = tipb_region_info->mutable_region_epoch(); + meta->set_conf_ver(region.second->confVer()); + meta->set_version(region.second->version()); + auto * range = tipb_region_info->add_ranges(); + auto handle_range = getHandleRangeByTable(region.second->getRange()->rawKeys(), table_id); + range->set_start(RecordKVFormat::genRawKey(table_id, handle_range.first.handle_id)); + range->set_end(RecordKVFormat::genRawKey(table_id, handle_range.second.handle_id)); +} + BlockInputStreamPtr executeQuery(Context & context, RegionID region_id, const DAGProperties & properties, QueryTasks & query_tasks, MakeResOutputStream & func_wrap_output_stream) { if (properties.is_mpp_query) @@ -288,22 +300,44 @@ BlockInputStreamPtr executeQuery(Context & context, RegionID region_id, const DA if (table_id != -1) { /// contains a table scan - auto regions = context.getTMTContext().getRegionTable().getRegionsByTable(table_id); - if (regions.size() < static_cast(properties.mpp_partition_num)) - throw Exception("Not supported: table region num less than mpp partition num"); - for (size_t i = 0; i < regions.size(); i++) + const auto & table_info = MockTiDB::instance().getTableInfoByID(table_id); + if (table_info->is_partition_table) { - if (i % properties.mpp_partition_num != static_cast(task.partition_id)) - continue; - auto * region = req->add_regions(); - region->set_region_id(regions[i].first); - auto * meta = region->mutable_region_epoch(); - meta->set_conf_ver(regions[i].second->confVer()); - meta->set_version(regions[i].second->version()); - auto * range = region->add_ranges(); - auto handle_range = getHandleRangeByTable(regions[i].second->getRange()->rawKeys(), table_id); - range->set_start(RecordKVFormat::genRawKey(table_id, handle_range.first.handle_id)); - range->set_end(RecordKVFormat::genRawKey(table_id, handle_range.second.handle_id)); + size_t current_region_size = 0; + coprocessor::TableRegions * current_table_regions = nullptr; + for (const auto & partition : table_info->partition.definitions) + { + const auto partition_id = partition.id; + auto regions = context.getTMTContext().getRegionTable().getRegionsByTable(partition_id); + for (size_t i = 0; i < regions.size(); ++i) + { + if ((current_region_size + i) % properties.mpp_partition_num != static_cast(task.partition_id)) + continue; + if (current_table_regions != nullptr && current_table_regions->physical_table_id() != partition_id) + current_table_regions = nullptr; + if (current_table_regions == nullptr) + { + current_table_regions = req->add_table_regions(); + current_table_regions->set_physical_table_id(partition_id); + } + setTipbRegionInfo(current_table_regions->add_regions(), regions[i], partition_id); + } + current_region_size += regions.size(); + } + if (current_region_size < static_cast(properties.mpp_partition_num)) + throw Exception("Not supported: table region num less than mpp partition num"); + } + else + { + auto regions = context.getTMTContext().getRegionTable().getRegionsByTable(table_id); + if (regions.size() < static_cast(properties.mpp_partition_num)) + throw Exception("Not supported: table region num less than mpp partition num"); + for (size_t i = 0; i < regions.size(); ++i) + { + if (i % properties.mpp_partition_num != static_cast(task.partition_id)) + continue; + setTipbRegionInfo(req->add_regions(), regions[i], table_id); + } } } pingcap::kv::RpcCall call(req); @@ -413,11 +447,12 @@ void dbgFuncTiDBQueryFromNaturalDag(Context & context, const ASTs & args, DBGInv static Poco::Logger * log = &Poco::Logger::get("MockDAG"); LOG_INFO(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); tipb::SelectResponse dag_response; - RegionInfoMap regions; - regions.emplace(region_id, RegionInfo(region_id, region->version(), region->confVer(), std::move(key_ranges), nullptr)); + TablesRegionsInfo tables_regions_info(true); + auto & table_regions_info = tables_regions_info.getSingleTableRegions(); + table_regions_info.local_regions.emplace(region_id, RegionInfo(region_id, region->version(), region->confVer(), std::move(key_ranges), nullptr)); DAGContext dag_context(dag_request); - dag_context.regions_for_local_read = regions; + dag_context.tables_regions_info = std::move(tables_regions_info); dag_context.log = std::make_shared(&Poco::Logger::get("MockDAG"), ""); context.setDAGContext(&dag_context); DAGDriver driver(context, properties.start_ts, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, &dag_response, true); @@ -1097,32 +1132,49 @@ struct TableScan : public Executor output_schema.erase(std::remove_if(output_schema.begin(), output_schema.end(), [&](const auto & field) { return used_columns.count(field.first) == 0; }), output_schema.end()); } - bool toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t, const MPPInfo &, const Context &) override + + void setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info) const { - tipb_executor->set_tp(tipb::ExecType::TypeTableScan); - tipb_executor->set_executor_id(name); - auto * ts = tipb_executor->mutable_tbl_scan(); - ts->set_table_id(table_info.id); - for (const auto & info : output_schema) - { - tipb::ColumnInfo * ci = ts->add_columns(); - auto column_name = splitQualifiedName(info.first).second; - if (column_name == MutableSupport::tidb_pk_column_name) - ci->set_column_id(-1); - else - ci->set_column_id(table_info.getColumnID(column_name)); - ci->set_tp(info.second.tp); - ci->set_flag(info.second.flag); - ci->set_columnlen(info.second.flen); - ci->set_decimal(info.second.decimal); - if (!info.second.elems.empty()) + auto column_name = splitQualifiedName(dag_column_info.first).second; + if (column_name == MutableSupport::tidb_pk_column_name) + ci->set_column_id(-1); + else + ci->set_column_id(table_info.getColumnID(column_name)); + ci->set_tp(dag_column_info.second.tp); + ci->set_flag(dag_column_info.second.flag); + ci->set_columnlen(dag_column_info.second.flen); + ci->set_decimal(dag_column_info.second.decimal); + if (!dag_column_info.second.elems.empty()) + { + for (const auto & pair : dag_column_info.second.elems) { - for (const auto & pair : info.second.elems) - { - ci->add_elems(pair.first); - } + ci->add_elems(pair.first); } } + } + + bool toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t, const MPPInfo &, const Context &) override + { + if (table_info.is_partition_table) + { + tipb_executor->set_tp(tipb::ExecType::TypePartitionTableScan); + tipb_executor->set_executor_id(name); + auto * partition_ts = tipb_executor->mutable_partition_table_scan(); + partition_ts->set_table_id(table_info.id); + for (const auto & info : output_schema) + setTipbColumnInfo(partition_ts->add_columns(), info); + for (const auto & partition : table_info.partition.definitions) + partition_ts->add_partition_ids(partition.id); + } + else + { + tipb_executor->set_tp(tipb::ExecType::TypeTableScan); + tipb_executor->set_executor_id(name); + auto * ts = tipb_executor->mutable_tbl_scan(); + ts->set_table_id(table_info.id); + for (const auto & info : output_schema) + setTipbColumnInfo(ts->add_columns(), info); + } return true; } void toMPPSubPlan(size_t &, const DAGProperties &, std::unordered_map, std::shared_ptr>> &) override @@ -2466,12 +2518,13 @@ tipb::SelectResponse executeDAGRequest(Context & context, const tipb::DAGRequest static Poco::Logger * log = &Poco::Logger::get("MockDAG"); LOG_DEBUG(log, __PRETTY_FUNCTION__ << ": Handling DAG request: " << dag_request.DebugString()); tipb::SelectResponse dag_response; - RegionInfoMap regions; + TablesRegionsInfo tables_regions_info(true); + auto & table_regions_info = tables_regions_info.getSingleTableRegions(); - regions.emplace(region_id, RegionInfo(region_id, region_version, region_conf_version, std::move(key_ranges), nullptr)); + table_regions_info.local_regions.emplace(region_id, RegionInfo(region_id, region_version, region_conf_version, std::move(key_ranges), nullptr)); DAGContext dag_context(dag_request); - dag_context.regions_for_local_read = regions; + dag_context.tables_regions_info = std::move(tables_regions_info); dag_context.log = std::make_shared(log, ""); context.setDAGContext(&dag_context); diff --git a/dbms/src/Flash/BatchCoprocessorHandler.cpp b/dbms/src/Flash/BatchCoprocessorHandler.cpp index 42b5ec97e70..4a520ebada0 100644 --- a/dbms/src/Flash/BatchCoprocessorHandler.cpp +++ b/dbms/src/Flash/BatchCoprocessorHandler.cpp @@ -43,33 +43,18 @@ grpc::Status BatchCoprocessorHandler::execute() { GET_METRIC(tiflash_coprocessor_handling_request_count, type_super_batch_cop_dag).Decrement(); }); auto dag_request = getDAGRequestFromStringWithRetry(cop_request->data()); - RegionInfoMap regions; - RegionInfoList retry_regions; - for (auto & r : cop_request->regions()) - { - auto res = regions.emplace(r.region_id(), - RegionInfo( - r.region_id(), - r.region_epoch().version(), - r.region_epoch().conf_ver(), - GenCopKeyRange(r.ranges()), - nullptr)); - if (!res.second) - { - retry_regions.emplace_back(RegionInfo(r.region_id(), r.region_epoch().version(), r.region_epoch().conf_ver(), CoprocessorHandler::GenCopKeyRange(r.ranges()), nullptr)); - } - } + auto tables_regions_info = TablesRegionsInfo::create(cop_request->regions(), cop_request->table_regions(), cop_context.db_context.getTMTContext()); LOG_FMT_DEBUG( log, - "{}: Handling {} regions in DAG request: {}", + "{}: Handling {} regions from {} physical tables in DAG request: {}", __PRETTY_FUNCTION__, - regions.size(), + tables_regions_info.regionCount(), + tables_regions_info.tableCount(), dag_request.DebugString()); DAGContext dag_context(dag_request); dag_context.is_batch_cop = true; - dag_context.regions_for_local_read = std::move(regions); - dag_context.regions_for_remote_read = std::move(retry_regions); + dag_context.tables_regions_info = std::move(tables_regions_info); dag_context.log = std::make_shared(log, ""); dag_context.tidb_host = cop_context.db_context.getClientInfo().current_address.toString(); cop_context.db_context.setDAGContext(&dag_context); diff --git a/dbms/src/Flash/BatchCoprocessorHandler.h b/dbms/src/Flash/BatchCoprocessorHandler.h index ca1d1166144..c2149a68b97 100644 --- a/dbms/src/Flash/BatchCoprocessorHandler.h +++ b/dbms/src/Flash/BatchCoprocessorHandler.h @@ -19,10 +19,10 @@ class BatchCoprocessorHandler : public CoprocessorHandler ~BatchCoprocessorHandler() = default; - grpc::Status execute(); + grpc::Status execute() override; protected: - grpc::Status recordError(grpc::StatusCode err_code, const String & err_msg); + grpc::Status recordError(grpc::StatusCode err_code, const String & err_msg) override; protected: const coprocessor::BatchRequest * cop_request; diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 49e35f02b88..7a012519445 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -209,4 +209,14 @@ int DAGContext::getNewThreadCountOfExchangeReceiver() const return new_thread_count_of_exchange_receiver; } +bool DAGContext::containsRegionsInfoForTable(Int64 table_id) const +{ + return tables_regions_info.containsRegionsInfoForTable(table_id); +} + +const SingleTableRegions & DAGContext::getTableRegionsInfoByTableID(Int64 table_id) const +{ + return tables_regions_info.getTableRegionInfoByTableID(table_id); +} + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index f7367a127ac..81fcd26cdb4 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -199,8 +200,9 @@ class DAGContext std::pair getTableScanThroughput(); - const RegionInfoMap & getRegionsForLocalRead() const { return regions_for_local_read; } - const RegionInfoList & getRegionsForRemoteRead() const { return regions_for_remote_read; } + const SingleTableRegions & getTableRegionsInfoByTableID(Int64 table_id) const; + + bool containsRegionsInfoForTable(Int64 table_id) const; const BlockIO & getBlockIO() const { @@ -229,6 +231,27 @@ class DAGContext return (flags & f); } + UInt64 getSQLMode() const + { + return sql_mode; + } + void setSQLMode(UInt64 f) + { + sql_mode = f; + } + void addSQLMode(UInt64 f) + { + sql_mode |= f; + } + void delSQLMode(UInt64 f) + { + sql_mode &= (~f); + } + bool hasSQLMode(UInt64 f) const + { + return sql_mode & f; + } + void initExchangeReceiverIfMPP(Context & context, size_t max_streams); const std::unordered_map> & getMPPExchangeReceiverMap() const; @@ -246,8 +269,7 @@ class DAGContext bool is_root_mpp_task = false; bool is_batch_cop = false; MPPTunnelSetPtr tunnel_set; - RegionInfoMap regions_for_local_read; - RegionInfoList regions_for_remote_read; + TablesRegionsInfo tables_regions_info; // part of regions_for_local_read + regions_for_remote_read, only used for batch-cop RegionInfoList retry_regions; diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 245ca1ca4d4..da9263e6f13 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -102,7 +102,7 @@ try if (!dag_context.retry_regions.empty()) { coprocessor::BatchResponse response; - for (auto region : dag_context.retry_regions) + for (const auto & region : dag_context.retry_regions) { auto * retry_region = response.add_retry_regions(); retry_region->set_id(region.region_id); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 1ef990f0889..78272c5199f 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -41,7 +41,7 @@ namespace { bool isUInt8Type(const DataTypePtr & type) { - return removeNullable(type)->getTypeId() == TypeIndex::UInt8; + return type->getTypeId() == TypeIndex::UInt8; } tipb::Expr constructTZExpr(const TimezoneInfo & dag_timezone_info) @@ -142,7 +142,7 @@ void DAGExpressionAnalyzer::fillAggArgumentDetail( { arg_names.push_back(getActions(arg, actions)); arg_types.push_back(actions->getSampleBlock().getByName(arg_names.back()).type); - arg_collators.push_back(removeNullable(arg_types.back())->isString() ? getCollatorFromExpr(arg) : nullptr); + arg_collators.push_back(arg_types.back()->isString() ? getCollatorFromExpr(arg) : nullptr); } void DAGExpressionAnalyzer::buildGroupConcat( @@ -306,7 +306,7 @@ void DAGExpressionAnalyzer::buildAggGroupBy( { auto type = actions->getSampleBlock().getByName(name).type; TiDB::TiDBCollatorPtr collator = nullptr; - if (removeNullable(type)->isString()) + if (type->isString()) collator = getCollatorFromExpr(expr); if (!duplicated_key) collators.push_back(collator); @@ -467,14 +467,14 @@ String DAGExpressionAnalyzer::convertToUInt8(const ExpressionActionsPtr & action { return column_name; } - const auto & org_type = removeNullable(actions->getSampleBlock().getByName(column_name).type); - if (org_type->isNumber() || org_type->isDecimal()) + const auto & type = actions->getSampleBlock().getByName(column_name).type; + if (type->isNumber() || type->isDecimal()) { tipb::Expr const_expr = constructInt64LiteralTiExpr(0); auto const_expr_name = getActions(const_expr, actions); return applyFunction("notEquals", {column_name, const_expr_name}, actions, nullptr); } - if (org_type->isStringOrFixedString()) + if (type->isStringOrFixedString()) { /// use tidb_cast to make it compatible with TiDB tipb::FieldType field_type; @@ -494,13 +494,13 @@ String DAGExpressionAnalyzer::convertToUInt8(const ExpressionActionsPtr & action auto const_expr_name = getActions(const_expr, actions); return applyFunction("notEquals", {num_col_name, const_expr_name}, actions, nullptr); } - if (org_type->isDateOrDateTime()) + if (type->isDateOrDateTime()) { tipb::Expr const_expr = constructDateTimeLiteralTiExpr(0); auto const_expr_name = getActions(const_expr, actions); return applyFunction("notEquals", {column_name, const_expr_name}, actions, nullptr); } - throw TiFlashException(fmt::format("Filter on {} is not supported.", org_type->getName()), Errors::Coprocessor::Unimplemented); + throw TiFlashException(fmt::format("Filter on {} is not supported.", type->getName()), Errors::Coprocessor::Unimplemented); } NamesAndTypes DAGExpressionAnalyzer::buildOrderColumns( @@ -1070,7 +1070,7 @@ String DAGExpressionAnalyzer::buildTupleFunctionForGroupConcat( argument_names.push_back(name); auto type = actions->getSampleBlock().getByName(name).type; names_and_types.emplace_back(name, type); - if (removeNullable(type)->isString()) + if (type->isString()) collators.push_back(getCollatorFromExpr(expr.children(i))); else collators.push_back(nullptr); @@ -1084,7 +1084,7 @@ String DAGExpressionAnalyzer::buildTupleFunctionForGroupConcat( auto type = actions->getSampleBlock().getByName(name).type; order_columns.emplace_back(name, type); names_and_types.emplace_back(name, type); - if (removeNullable(type)->isString()) + if (type->isString()) collators.push_back(getCollatorFromExpr(expr.order_by(i).expr())); else collators.push_back(nullptr); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp index 397be528c70..c561c351c74 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp @@ -288,7 +288,7 @@ String DAGExpressionAnalyzerHelper::buildDateAddOrSubFunction( fmt::format("{} function does not support unit {} yet.", Impl::name, unit), Errors::Coprocessor::Unimplemented); String func_name = Impl::unit_to_func_name_map.find(unit)->second; - const auto & delta_column_type = removeNullable(actions->getSampleBlock().getByName(delta_column).type); + const auto & delta_column_type = actions->getSampleBlock().getByName(delta_column).type; if (!delta_column_type->isNumber()) { // convert to numeric first @@ -326,7 +326,7 @@ String DAGExpressionAnalyzerHelper::buildBitwiseFunction( DataTypePtr orig_type = sample_block.getByName(name).type; // Bump argument type - if (!removeNullable(orig_type)->equals(*uint64_type)) + if (!orig_type->equals(*uint64_type)) { if (orig_type->isNullable()) { diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp index b4ee18ad44d..4a2e3815dc5 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.cpp @@ -22,7 +22,8 @@ class Context; bool isSourceNode(const tipb::Executor * root) { return root->tp() == tipb::ExecType::TypeJoin || root->tp() == tipb::ExecType::TypeTableScan - || root->tp() == tipb::ExecType::TypeExchangeReceiver || root->tp() == tipb::ExecType::TypeProjection; + || root->tp() == tipb::ExecType::TypeExchangeReceiver || root->tp() == tipb::ExecType::TypeProjection + || root->tp() == tipb::ExecType::TypePartitionTableScan; } const static String SOURCE_NAME("source"); @@ -132,6 +133,10 @@ DAGQueryBlock::DAGQueryBlock(const tipb::Executor & root_, QueryBlockIDGenerator { GET_METRIC(tiflash_coprocessor_executor_count, type_ts).Increment(); } + else if (current->tp() == tipb::ExecType::TypePartitionTableScan) + { + GET_METRIC(tiflash_coprocessor_executor_count, type_partition_ts).Increment(); + } } /// construct DAGQueryBlock from a list struct based executors, which is the diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h index eab642f0397..6dd0a77581c 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlock.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlock.h @@ -59,6 +59,7 @@ class DAGQueryBlock std::vector output_offsets; bool isRootQueryBlock() const { return id == 1; }; + bool isTableScanSource() const { return source->tp() == tipb::ExecType::TypeTableScan || source->tp() == tipb::ExecType::TypePartitionTableScan; } }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 5ff09d6d8ca..639ef0df5fc 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include @@ -111,7 +110,7 @@ AnalysisResult analyzeExpressions( AnalysisResult res; ExpressionActionsChain chain; // selection on table scan had been executed in handleTableScan - if (query_block.selection && query_block.source->tp() != tipb::ExecType::TypeTableScan) + if (query_block.selection && !query_block.isTableScanSource()) { std::vector where_conditions; for (const auto & c : query_block.selection->selection().conditions()) @@ -230,20 +229,20 @@ ExpressionActionsPtr generateProjectExpressionActions( return project; } -void DAGQueryBlockInterpreter::handleTableScan(const tipb::TableScan & ts, DAGPipeline & pipeline) +void DAGQueryBlockInterpreter::handleTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline) { - if (!ts.has_table_id()) + bool has_region_to_read = false; + for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) { - // do not have table id - throw TiFlashException("Table id not specified in table scan executor", Errors::Coprocessor::BadRequest); - } - if (dagContext().getRegionsForLocalRead().empty() && dagContext().getRegionsForRemoteRead().empty()) - { - throw TiFlashException("Dag Request does not have region to read. ", Errors::Coprocessor::BadRequest); + const auto & table_regions_info = dagContext().getTableRegionsInfoByTableID(physical_table_id); + if (!table_regions_info.local_regions.empty() || !table_regions_info.remote_regions.empty()) + { + has_region_to_read = true; + break; + } } - if (ts.next_read_engine() != tipb::EngineType::Local) - throw TiFlashException("Unsupported remote query.", Errors::Coprocessor::BadRequest); - + if (!has_region_to_read) + throw TiFlashException(fmt::format("Dag Request does not have region to read for table: {}", table_scan.getLogicalTableID()), Errors::Coprocessor::BadRequest); // construct pushed down filter conditions. std::vector conditions; if (query_block.selection) @@ -252,19 +251,13 @@ void DAGQueryBlockInterpreter::handleTableScan(const tipb::TableScan & ts, DAGPi conditions.push_back(&condition); } - DAGStorageInterpreter storage_interpreter(context, query_block, ts, conditions, max_streams); + DAGStorageInterpreter storage_interpreter(context, query_block, table_scan, conditions, max_streams); storage_interpreter.execute(pipeline); analyzer = std::move(storage_interpreter.analyzer); - // The DeltaTree engine ensures that once input streams are created, the caller can get a consistent result - // from those streams even if DDL operations are applied. Release the alter lock so that reading does not - // block DDL operations, keep the drop lock so that the storage not to be dropped during reading. - std::tie(std::ignore, table_drop_lock) = std::move(storage_interpreter.table_structure_lock).release(); - auto region_retry = std::move(storage_interpreter.region_retry); - auto dag_req = std::move(storage_interpreter.dag_request); - auto schema = std::move(storage_interpreter.dag_schema); + auto remote_requests = std::move(storage_interpreter.remote_requests); auto null_stream_if_empty = std::move(storage_interpreter.null_stream_if_empty); // It is impossible to have no joined stream. @@ -273,23 +266,8 @@ void DAGQueryBlockInterpreter::handleTableScan(const tipb::TableScan & ts, DAGPi size_t remote_read_streams_start_index = pipeline.streams.size(); // For those regions which are not presented in this tiflash node, we will try to fetch streams by key ranges from other tiflash nodes, only happens in batch cop / mpp mode. - if (!region_retry.empty()) - { -#ifndef NDEBUG - if (unlikely(!dag_req.has_value() || !schema.has_value())) - throw TiFlashException( - "Try to read from remote but can not build DAG request. Should not happen!", - Errors::Coprocessor::Internal); -#endif - std::vector ranges; - for (auto & info : region_retry) - { - for (const auto & range : info.get().key_ranges) - ranges.emplace_back(*range.first, *range.second); - } - sort(ranges.begin(), ranges.end()); - executeRemoteQueryImpl(pipeline, ranges, *dag_req, *schema); - } + if (!remote_requests.empty()) + executeRemoteQueryImpl(pipeline, remote_requests); /// record local and remote io input stream auto & table_scan_io_input_streams = dagContext().getInBoundIOInputStreamsMap()[query_block.source_name]; @@ -302,7 +280,15 @@ void DAGQueryBlockInterpreter::handleTableScan(const tipb::TableScan & ts, DAGPi remote_read_streams_start_index = 1; } - pipeline.transform([&](auto & stream) { stream->addTableLock(table_drop_lock); }); + /// Theoretically we could move addTableLock to DAGStorageInterpreter, but we don't wants to the table to be dropped + /// during the lifetime of this query, and sometimes if there is no local region, we will use the RemoteBlockInputStream + /// or even the null_stream to hold the lock, so I would like too keep the addTableLock in DAGQueryBlockInterpreter + pipeline.transform([&](auto & stream) { + // todo do not need to hold all locks in each stream, if the stream is reading from table a + // it only needs to hold the lock of table a + for (auto & lock : storage_interpreter.drop_locks) + stream->addTableLock(lock); + }); /// Set the limits and quota for reading data, the speed and time of the query. setQuotaAndLimitsOnTableScan(context, pipeline); @@ -617,7 +603,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & if (join.probe_types_size() == static_cast(join_key_size) && join.build_types_size() == join.probe_types_size()) for (size_t i = 0; i < join_key_size; i++) { - if (removeNullable(join_key_types[i])->isString()) + if (join_key_types[i]->isString()) { if (join.probe_types(i).collate() != join.build_types(i).collate()) throw TiFlashException("Join with different collators on the join key", Errors::Coprocessor::BadRequest); @@ -888,23 +874,51 @@ void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, cons pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); } +bool schemaMatch(const DAGSchema & left, const DAGSchema & right) +{ + if (left.size() != right.size()) + return false; + for (size_t i = 0; i < left.size(); i++) + { + const auto & left_ci = left[i]; + const auto & right_ci = right[i]; + if (left_ci.second.tp != right_ci.second.tp) + return false; + if (left_ci.second.flag != right_ci.second.flag) + return false; + } + return true; +} + void DAGQueryBlockInterpreter::executeRemoteQueryImpl( DAGPipeline & pipeline, - const std::vector & cop_key_ranges, - ::tipb::DAGRequest & dag_req, - const DAGSchema & schema) + std::vector & remote_requests) { - pingcap::coprocessor::RequestPtr req = std::make_shared(); - dag_req.SerializeToString(&(req->data)); - req->tp = pingcap::coprocessor::ReqType::DAG; - req->start_ts = context.getSettingsRef().read_tso; - req->schema_version = context.getSettingsRef().schema_version; - bool has_enforce_encode_type = dag_req.has_force_encode_type() && dag_req.force_encode_type(); - + assert(!remote_requests.empty()); + DAGSchema & schema = remote_requests[0].schema; +#ifndef NDEBUG + for (size_t i = 1; i < remote_requests.size(); i++) + { + if (!schemaMatch(schema, remote_requests[i].schema)) + throw Exception("Schema mismatch between different partitions for partition table"); + } +#endif + bool has_enforce_encode_type = remote_requests[0].dag_request.has_force_encode_type() && remote_requests[0].dag_request.force_encode_type(); pingcap::kv::Cluster * cluster = context.getTMTContext().getKVCluster(); - pingcap::kv::Backoffer bo(pingcap::kv::copBuildTaskMaxBackoff); - pingcap::kv::StoreType store_type = pingcap::kv::StoreType::TiFlash; - auto all_tasks = pingcap::coprocessor::buildCopTasks(bo, cluster, cop_key_ranges, req, store_type, &Poco::Logger::get("pingcap/coprocessor")); + std::vector all_tasks; + for (const auto & remote_request : remote_requests) + { + pingcap::coprocessor::RequestPtr req = std::make_shared(); + remote_request.dag_request.SerializeToString(&(req->data)); + req->tp = pingcap::coprocessor::ReqType::DAG; + req->start_ts = context.getSettingsRef().read_tso; + req->schema_version = context.getSettingsRef().schema_version; + + pingcap::kv::Backoffer bo(pingcap::kv::copBuildTaskMaxBackoff); + pingcap::kv::StoreType store_type = pingcap::kv::StoreType::TiFlash; + auto tasks = pingcap::coprocessor::buildCopTasks(bo, cluster, remote_request.key_ranges, req, store_type, &Poco::Logger::get("pingcap/coprocessor")); + all_tasks.insert(all_tasks.end(), tasks.begin(), tasks.end()); + } size_t concurrent_num = std::min(context.getSettingsRef().max_threads, all_tasks.size()); size_t task_per_thread = all_tasks.size() / concurrent_num; @@ -1006,11 +1020,18 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) handleProjection(pipeline, query_block.source->projection()); recordProfileStreams(pipeline, query_block.source_name); } - else + else if (query_block.isTableScanSource()) { - handleTableScan(query_block.source->tbl_scan(), pipeline); + TiDBTableScan table_scan(query_block.source, dagContext()); + handleTableScan(table_scan, pipeline); dagContext().table_scan_executor_id = query_block.source_name; } + else + { + throw TiFlashException( + std::string(__PRETTY_FUNCTION__) + ": Unsupported source node: " + query_block.source_name, + Errors::Coprocessor::BadRequest); + } auto res = analyzeExpressions( context, @@ -1032,7 +1053,7 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) "execution stream size for query block(before aggregation) {} is {}", query_block.qb_column_prefix, pipeline.streams.size()); - dagContext().final_concurrency = std::max(dagContext().final_concurrency, pipeline.streams.size()); + dagContext().final_concurrency = std::min(std::max(dagContext().final_concurrency, pipeline.streams.size()), max_streams); if (res.before_aggregation) { @@ -1126,7 +1147,7 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) assert(isColumnExpr(expr)); auto column_index = decodeDAGInt64(expr.val()); partition_col_id.emplace_back(column_index); - if (has_collator_info && removeNullable(getDataTypeByFieldTypeForComputingLayer(expr.field_type()))->isString()) + if (has_collator_info && getDataTypeByFieldTypeForComputingLayer(expr.field_type())->isString()) { collators.emplace_back(getCollatorFromFieldType(exchange_sender.types(i))); } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 9c697a003aa..7f8ba2a5e34 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -43,7 +44,7 @@ class DAGQueryBlockInterpreter private: void executeImpl(DAGPipeline & pipeline); - void handleTableScan(const tipb::TableScan & ts, DAGPipeline & pipeline); + void handleTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline); void executeCastAfterTableScan(const std::vector & is_need_add_cast_column, size_t remote_read_streams_start_index, DAGPipeline & pipeline); void executePushedDownFilter(const std::vector & conditions, size_t remote_read_streams_start_index, DAGPipeline & pipeline); void handleJoin(const tipb::Join & join, DAGPipeline & pipeline, SubqueryForSet & right_query); @@ -85,9 +86,7 @@ class DAGQueryBlockInterpreter void executeRemoteQueryImpl( DAGPipeline & pipeline, - const std::vector & cop_key_ranges, - ::tipb::DAGRequest & dag_req, - const DAGSchema & schema); + std::vector & remote_requests); DAGContext & dagContext() const { return *context.getDAGContext(); } const LogWithPrefixPtr & taskLogger() const { return dagContext().log; } @@ -102,8 +101,6 @@ class DAGQueryBlockInterpreter /// How many streams we ask for storage to produce, and in how many threads we will do further processing. size_t max_streams = 1; - TableLockHolder table_drop_lock; - std::unique_ptr analyzer; std::vector & subqueries_for_sets; diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 7900cd41b26..7b578d3245e 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -44,68 +44,67 @@ RegionException::RegionReadStatus GetRegionReadStatus( std::tuple, RegionException::RegionReadStatus> MakeRegionQueryInfos( - const RegionInfoMap & dag_region_infos, + const TablesRegionInfoMap & dag_region_infos, const std::unordered_set & region_force_retry, TMTContext & tmt, MvccQueryInfo & mvcc_info, - TableID table_id, bool batch_cop [[maybe_unused]]) { mvcc_info.regions_query_info.clear(); RegionRetryList region_need_retry; RegionException::RegionReadStatus status_res = RegionException::RegionReadStatus::OK; - for (const auto & [id, r] : dag_region_infos) + for (const auto & [physical_table_id, regions] : dag_region_infos) { - if (r.key_ranges.empty()) + for (const auto & [id, r] : regions.get()) { - throw TiFlashException( - "Income key ranges is empty for region: " + std::to_string(r.region_id), - Errors::Coprocessor::BadRequest); - } - if (region_force_retry.count(id)) - { - region_need_retry.emplace_back(r); - status_res = RegionException::RegionReadStatus::NOT_FOUND; - continue; - } - ImutRegionRangePtr region_range{nullptr}; - auto status = GetRegionReadStatus(r, tmt.getKVStore()->getRegion(id), region_range); - fiu_do_on(FailPoints::force_remote_read_for_batch_cop, { - if (batch_cop) - status = RegionException::RegionReadStatus::NOT_FOUND; - }); - if (status != RegionException::RegionReadStatus::OK) - { - region_need_retry.emplace_back(r); - status_res = status; - continue; - } - RegionQueryInfo info; - { - info.region_id = id; - info.version = r.region_version; - info.conf_version = r.region_conf_version; - info.range_in_table = region_range->rawKeys(); - for (const auto & p : r.key_ranges) + if (r.key_ranges.empty()) + { + throw TiFlashException( + "Income key ranges is empty for region: " + std::to_string(r.region_id), + Errors::Coprocessor::BadRequest); + } + if (region_force_retry.count(id)) + { + region_need_retry.emplace_back(r); + status_res = RegionException::RegionReadStatus::NOT_FOUND; + continue; + } + ImutRegionRangePtr region_range{nullptr}; + auto status = GetRegionReadStatus(r, tmt.getKVStore()->getRegion(id), region_range); + fiu_do_on(FailPoints::force_remote_read_for_batch_cop, { + if (batch_cop) + status = RegionException::RegionReadStatus::NOT_FOUND; + }); + if (status != RegionException::RegionReadStatus::OK) + { + region_need_retry.emplace_back(r); + status_res = status; + continue; + } + RegionQueryInfo info(id, r.region_version, r.region_conf_version, physical_table_id); { - TableID table_id_in_range = -1; - if (!computeMappedTableID(*p.first, table_id_in_range) || table_id_in_range != table_id) + info.range_in_table = region_range->rawKeys(); + for (const auto & p : r.key_ranges) { - throw TiFlashException( - "Income key ranges is illegal for region: " + std::to_string(r.region_id) - + ", table id in key range is " + std::to_string(table_id_in_range) + ", table id in region is " - + std::to_string(table_id), - Errors::Coprocessor::BadRequest); + TableID table_id_in_range = -1; + if (!computeMappedTableID(*p.first, table_id_in_range) || table_id_in_range != physical_table_id) + { + throw TiFlashException( + "Income key ranges is illegal for region: " + std::to_string(r.region_id) + + ", table id in key range is " + std::to_string(table_id_in_range) + ", table id in region is " + + std::to_string(physical_table_id), + Errors::Coprocessor::BadRequest); + } + if (p.first->compare(*info.range_in_table.first) < 0 || p.second->compare(*info.range_in_table.second) > 0) + throw TiFlashException( + "Income key ranges is illegal for region: " + std::to_string(r.region_id), + Errors::Coprocessor::BadRequest); } - if (p.first->compare(*info.range_in_table.first) < 0 || p.second->compare(*info.range_in_table.second) > 0) - throw TiFlashException( - "Income key ranges is illegal for region: " + std::to_string(r.region_id), - Errors::Coprocessor::BadRequest); + info.required_handle_ranges = r.key_ranges; + info.bypass_lock_ts = r.bypass_lock_ts; } - info.required_handle_ranges = r.key_ranges; - info.bypass_lock_ts = r.bypass_lock_ts; + mvcc_info.regions_query_info.emplace_back(std::move(info)); } - mvcc_info.regions_query_info.emplace_back(std::move(info)); } mvcc_info.concurrent = mvcc_info.regions_query_info.size() > 1 ? 1.0 : 0.0; @@ -120,16 +119,16 @@ MakeRegionQueryInfos( DAGStorageInterpreter::DAGStorageInterpreter( Context & context_, const DAGQueryBlock & query_block_, - const tipb::TableScan & ts, + const TiDBTableScan & table_scan_, const std::vector & conditions_, size_t max_streams_) : context(context_) , query_block(query_block_) - , table_scan(ts) + , table_scan(table_scan_) , conditions(conditions_) , max_streams(max_streams_) , log(getMPPTaskLog(*context.getDAGContext(), "DAGStorageInterpreter")) - , table_id(ts.table_id()) + , logical_table_id(table_scan.getLogicalTableID()) , settings(context.getSettingsRef()) , tmt(context.getTMTContext()) , mvcc_query_info(new MvccQueryInfo(true, settings.read_tso)) @@ -144,9 +143,11 @@ void DAGStorageInterpreter::execute(DAGPipeline & pipeline) else learner_read_snapshot = doCopLearnerRead(); - std::tie(storage, table_structure_lock) = getAndLockStorage(settings.schema_version); + storages_with_structure_lock = getAndLockStorages(settings.schema_version); + assert(storages_with_structure_lock.find(logical_table_id) != storages_with_structure_lock.end()); + storage_for_logical_table = storages_with_structure_lock[logical_table_id].storage; - std::tie(required_columns, source_columns, is_need_add_cast_column, handle_column_name) = getColumnsForTableScan(settings.max_columns_to_read); + std::tie(required_columns, source_columns, is_need_add_cast_column) = getColumnsForTableScan(settings.max_columns_to_read); analyzer = std::make_unique(std::move(source_columns), context); @@ -155,35 +156,47 @@ void DAGStorageInterpreter::execute(DAGPipeline & pipeline) if (!mvcc_query_info->regions_query_info.empty()) doLocalRead(pipeline, settings.max_block_size); - for (const auto & region_info : dag_context.getRegionsForRemoteRead()) - region_retry.emplace_back(region_info); - - null_stream_if_empty = std::make_shared(storage->getSampleBlockForColumns(required_columns)); + null_stream_if_empty = std::make_shared(storage_for_logical_table->getSampleBlockForColumns(required_columns)); // Should build these vars under protect of `table_structure_lock`. - std::tie(dag_request, dag_schema) = buildRemoteTS(); + buildRemoteRequests(); + + releaseAlterLocks(); } LearnerReadSnapshot DAGStorageInterpreter::doCopLearnerRead() { + if (table_scan.isPartitionTableScan()) + { + throw Exception("Cop request does not support partition table scan"); + } + TablesRegionInfoMap regions_for_local_read; + for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) + { + regions_for_local_read.emplace(physical_table_id, std::cref(context.getDAGContext()->getTableRegionsInfoByTableID(physical_table_id).local_regions)); + } auto [info_retry, status] = MakeRegionQueryInfos( - context.getDAGContext()->getRegionsForLocalRead(), + regions_for_local_read, {}, tmt, *mvcc_query_info, - table_id, false); if (info_retry) throw RegionException({info_retry->begin()->get().region_id}, status); - return doLearnerRead(table_id, *mvcc_query_info, max_streams, /*wait_index_timeout_as_region_not_found*/ true, context, log); + return doLearnerRead(logical_table_id, *mvcc_query_info, max_streams, false, context, log); } -/// Will assign region_retry +/// Will assign region_retry_from_local_region LearnerReadSnapshot DAGStorageInterpreter::doBatchCopLearnerRead() { - const auto & regions_for_local_read = context.getDAGContext()->getRegionsForLocalRead(); + TablesRegionInfoMap regions_for_local_read; + for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) + { + const auto & local_regions = context.getDAGContext()->getTableRegionsInfoByTableID(physical_table_id).local_regions; + regions_for_local_read.emplace(physical_table_id, std::cref(local_regions)); + } if (regions_for_local_read.empty()) return {}; std::unordered_set force_retry; @@ -191,25 +204,24 @@ LearnerReadSnapshot DAGStorageInterpreter::doBatchCopLearnerRead() { try { - region_retry.clear(); + region_retry_from_local_region.clear(); auto [retry, status] = MakeRegionQueryInfos( regions_for_local_read, force_retry, tmt, *mvcc_query_info, - table_id, true); UNUSED(status); if (retry) { - region_retry = std::move(*retry); - for (const auto & r : region_retry) + region_retry_from_local_region = std::move(*retry); + for (const auto & r : region_retry_from_local_region) force_retry.emplace(r.get().region_id); } if (mvcc_query_info->regions_query_info.empty()) return {}; - return doLearnerRead(table_id, *mvcc_query_info, max_streams, /*wait_index_timeout_as_region_not_found*/ false, context, log); + return doLearnerRead(logical_table_id, *mvcc_query_info, max_streams, true, context, log); } catch (const LockException & e) { @@ -227,175 +239,245 @@ LearnerReadSnapshot DAGStorageInterpreter::doBatchCopLearnerRead() } catch (DB::Exception & e) { - e.addMessage("(while doing learner read for table, table_id: " + DB::toString(table_id) + ")"); + e.addMessage("(while doing learner read for table, logical table_id: " + DB::toString(logical_table_id) + ")"); throw; } } } +std::unordered_map DAGStorageInterpreter::generateSelectQueryInfos() +{ + std::unordered_map ret; + auto create_query_info = [&]() -> SelectQueryInfo { + SelectQueryInfo query_info; + /// to avoid null point exception + query_info.query = makeDummyQuery(); + query_info.dag_query = std::make_unique( + conditions, + analyzer->getPreparedSets(), + analyzer->getCurrentInputColumns(), + context.getTimezoneInfo()); + return query_info; + }; + if (table_scan.isPartitionTableScan()) + { + for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) + { + SelectQueryInfo query_info = create_query_info(); + query_info.mvcc_query_info = std::make_unique(mvcc_query_info->resolve_locks, mvcc_query_info->read_tso); + ret.emplace(physical_table_id, std::move(query_info)); + } + for (auto & r : mvcc_query_info->regions_query_info) + { + ret[r.physical_table_id].mvcc_query_info->regions_query_info.push_back(r); + } + for (auto & p : ret) + { + // todo mvcc_query_info->concurrent is not used anymore, should remove it later + p.second.mvcc_query_info->concurrent = p.second.mvcc_query_info->regions_query_info.size() > 1 ? 1.0 : 0.0; + } + } + else + { + TableID table_id = logical_table_id; + SelectQueryInfo query_info = create_query_info(); + query_info.mvcc_query_info = std::move(mvcc_query_info); + ret.emplace(table_id, std::move(query_info)); + } + return ret; +} + void DAGStorageInterpreter::doLocalRead(DAGPipeline & pipeline, size_t max_block_size) { const DAGContext & dag_context = *context.getDAGContext(); - SelectQueryInfo query_info; - /// to avoid null point exception - query_info.query = makeDummyQuery(); - query_info.dag_query = std::make_unique( - conditions, - analyzer->getPreparedSets(), - analyzer->getCurrentInputColumns(), - context.getTimezoneInfo()); - query_info.mvcc_query_info = std::move(mvcc_query_info); - - QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; - // TODO: Note that if storage is (Txn)MergeTree, and any region exception thrown, we won't do retry here. - // Now we only support DeltaTree in production environment and don't do any extra check for storage type here. - - int num_allow_retry = 1; - while (true) + size_t total_local_region_num = mvcc_query_info->regions_query_info.size(); + if (total_local_region_num == 0) + return; + auto table_query_infos = generateSelectQueryInfos(); + for (auto & table_query_info : table_query_infos) { - try + DAGPipeline current_pipeline; + TableID table_id = table_query_info.first; + SelectQueryInfo & query_info = table_query_info.second; + size_t region_num = query_info.mvcc_query_info->regions_query_info.size(); + if (region_num == 0) + continue; + /// calculate weighted max_streams for each partition, note at least 1 stream is needed for each partition + size_t current_max_streams = table_query_infos.size() == 1 ? max_streams : (max_streams * region_num + total_local_region_num - 1) / total_local_region_num; + + QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; + assert(storages_with_structure_lock.find(table_id) != storages_with_structure_lock.end()); + auto & storage = storages_with_structure_lock[table_id].storage; + + int num_allow_retry = 1; + while (true) { - pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + try + { + current_pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, current_max_streams); - // After getting streams from storage, we need to validate whether regions have changed or not after learner read. - // In case the versions of regions have changed, those `streams` may contain different data other than expected. - // Like after region merge/split. + // After getting streams from storage, we need to validate whether regions have changed or not after learner read. + // In case the versions of regions have changed, those `streams` may contain different data other than expected. + // Like after region merge/split. - // Inject failpoint to throw RegionException - fiu_do_on(FailPoints::region_exception_after_read_from_storage_some_error, { - const auto & regions_info = query_info.mvcc_query_info->regions_query_info; - RegionException::UnavailableRegions region_ids; - for (const auto & info : regions_info) - { - if (random() % 100 > 50) + // Inject failpoint to throw RegionException + fiu_do_on(FailPoints::region_exception_after_read_from_storage_some_error, { + const auto & regions_info = query_info.mvcc_query_info->regions_query_info; + RegionException::UnavailableRegions region_ids; + for (const auto & info : regions_info) + { + if (random() % 100 > 50) + region_ids.insert(info.region_id); + } + throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); + }); + fiu_do_on(FailPoints::region_exception_after_read_from_storage_all_error, { + const auto & regions_info = query_info.mvcc_query_info->regions_query_info; + RegionException::UnavailableRegions region_ids; + for (const auto & info : regions_info) region_ids.insert(info.region_id); - } - throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); - }); - fiu_do_on(FailPoints::region_exception_after_read_from_storage_all_error, { - const auto & regions_info = query_info.mvcc_query_info->regions_query_info; - RegionException::UnavailableRegions region_ids; - for (const auto & info : regions_info) - region_ids.insert(info.region_id); - throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); - }); - validateQueryInfo(*query_info.mvcc_query_info, learner_read_snapshot, tmt, log); - break; - } - catch (RegionException & e) - { - /// Recover from region exception when super batch is enable - if (dag_context.isBatchCop() || dag_context.isMPPTask()) + throw RegionException(std::move(region_ids), RegionException::RegionReadStatus::NOT_FOUND); + }); + validateQueryInfo(*query_info.mvcc_query_info, learner_read_snapshot, tmt, log); + break; + } + catch (RegionException & e) { - // clean all streams from local because we are not sure the correctness of those streams - pipeline.streams.clear(); - const auto & dag_regions = dag_context.getRegionsForLocalRead(); - FmtBuffer buffer; - // Normally there is only few regions need to retry when super batch is enabled. Retry to read - // from local first. However, too many retry in different places may make the whole process - // time out of control. We limit the number of retries to 1 now. - if (likely(num_allow_retry > 0)) + /// Recover from region exception when super batch is enable + if (dag_context.isBatchCop() || dag_context.isMPPTask()) { - --num_allow_retry; - auto & regions_query_info = query_info.mvcc_query_info->regions_query_info; - for (auto iter = regions_query_info.begin(); iter != regions_query_info.end(); /**/) + // clean all streams from local because we are not sure the correctness of those streams + current_pipeline.streams.clear(); + const auto & dag_regions = dag_context.getTableRegionsInfoByTableID(table_id).local_regions; + FmtBuffer buffer; + // Normally there is only few regions need to retry when super batch is enabled. Retry to read + // from local first. However, too many retry in different places may make the whole process + // time out of control. We limit the number of retries to 1 now. + if (likely(num_allow_retry > 0)) { - if (e.unavailable_region.find(iter->region_id) != e.unavailable_region.end()) + --num_allow_retry; + auto & regions_query_info = query_info.mvcc_query_info->regions_query_info; + for (auto iter = regions_query_info.begin(); iter != regions_query_info.end(); /**/) { - // move the error regions info from `query_info.mvcc_query_info->regions_query_info` to `region_retry` - if (auto region_iter = dag_regions.find(iter->region_id); likely(region_iter != dag_regions.end())) + if (e.unavailable_region.find(iter->region_id) != e.unavailable_region.end()) { - region_retry.emplace_back(region_iter->second); - buffer.fmtAppend("{},", region_iter->first); + // move the error regions info from `query_info.mvcc_query_info->regions_query_info` to `region_retry_from_local_region` + if (auto region_iter = dag_regions.find(iter->region_id); likely(region_iter != dag_regions.end())) + { + region_retry_from_local_region.emplace_back(region_iter->second); + buffer.fmtAppend("{},", region_iter->first); + } + iter = regions_query_info.erase(iter); + } + else + { + ++iter; } - iter = regions_query_info.erase(iter); } - else + LOG_FMT_WARNING( + log, + "RegionException after read from storage, regions [{}], message: {}{}", + buffer.toString(), + e.message(), + (regions_query_info.empty() ? "" : ", retry to read from local")); + if (unlikely(regions_query_info.empty())) + break; // no available region in local, break retry loop + continue; // continue to retry read from local storage + } + else + { + // push all regions to `region_retry_from_local_region` to retry from other tiflash nodes + for (const auto & region : query_info.mvcc_query_info->regions_query_info) { - ++iter; + auto iter = dag_regions.find(region.region_id); + if (likely(iter != dag_regions.end())) + { + region_retry_from_local_region.emplace_back(iter->second); + buffer.fmtAppend("{},", iter->first); + } } + LOG_FMT_WARNING(log, "RegionException after read from storage, regions [{}], message: {}", buffer.toString(), e.message()); + break; // break retry loop } - LOG_FMT_WARNING( - log, - "RegionException after read from storage, regions [{}], message: {}{}", - buffer.toString(), - e.message(), - (regions_query_info.empty() ? "" : ", retry to read from local")); - if (unlikely(regions_query_info.empty())) - break; // no available region in local, break retry loop - continue; // continue to retry read from local storage } else { - // push all regions to `region_retry` to retry from other tiflash nodes - for (const auto & region : query_info.mvcc_query_info->regions_query_info) - { - auto iter = dag_regions.find(region.region_id); - if (likely(iter != dag_regions.end())) - { - region_retry.emplace_back(iter->second); - buffer.fmtAppend("{},", iter->first); - } - } - LOG_FMT_WARNING(log, "RegionException after read from storage, regions [{}], message: {}", buffer.toString(), e.message()); - break; // break retry loop + // Throw an exception for TiDB / TiSpark to retry + if (table_id == logical_table_id) + e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() + + "`, table_id: " + DB::toString(table_id) + ")"); + else + e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() + + "`, table_id: " + DB::toString(table_id) + ", logical_table_id: " + DB::toString(logical_table_id) + ")"); + throw; } } - else + catch (DB::Exception & e) { - // Throw an exception for TiDB / TiSpark to retry - e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() - + "`, table_id: " + DB::toString(table_id) + ")"); + /// Other unknown exceptions + if (table_id == logical_table_id) + e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() + + "`, table_id: " + DB::toString(table_id) + ")"); + else + e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() + + "`, table_id: " + DB::toString(table_id) + ", logical_table_id: " + DB::toString(logical_table_id) + ")"); throw; } } - catch (DB::Exception & e) - { - /// Other unknown exceptions - e.addMessage("(while creating InputStreams from storage `" + storage->getDatabaseName() + "`.`" + storage->getTableName() - + "`, table_id: " + DB::toString(table_id) + ")"); - throw; - } + pipeline.streams.insert(pipeline.streams.end(), current_pipeline.streams.begin(), current_pipeline.streams.end()); } } -std::tuple DAGStorageInterpreter::getAndLockStorage(Int64 query_schema_version) +std::unordered_map DAGStorageInterpreter::getAndLockStorages(Int64 query_schema_version) { - /// Get current schema version in schema syncer for a chance to shortcut. + std::unordered_map storages_with_lock; if (unlikely(query_schema_version == DEFAULT_UNSPECIFIED_SCHEMA_VERSION)) { - auto managed_storage = tmt.getStorages().get(table_id); - if (!managed_storage) + auto logical_table_storage = tmt.getStorages().get(logical_table_id); + if (!logical_table_storage) { - throw TiFlashException("Table " + std::to_string(table_id) + " doesn't exist.", Errors::Table::NotExists); + throw TiFlashException("Table " + std::to_string(logical_table_id) + " doesn't exist.", Errors::Table::NotExists); } - return {managed_storage, managed_storage->lockStructureForShare(context.getCurrentQueryId())}; + storages_with_lock[logical_table_id] = {logical_table_storage, logical_table_storage->lockStructureForShare(context.getCurrentQueryId())}; + if (table_scan.isPartitionTableScan()) + { + for (auto const physical_table_id : table_scan.getPhysicalTableIDs()) + { + auto physical_table_storage = tmt.getStorages().get(physical_table_id); + if (!physical_table_storage) + { + throw TiFlashException("Table " + std::to_string(physical_table_id) + " doesn't exist.", Errors::Table::NotExists); + } + storages_with_lock[physical_table_id] = {physical_table_storage, physical_table_storage->lockStructureForShare(context.getCurrentQueryId())}; + } + } + return storages_with_lock; } auto global_schema_version = tmt.getSchemaSyncer()->getCurrentVersion(); /// Align schema version under the read lock. /// Return: [storage, table_structure_lock, storage_schema_version, ok] - auto get_and_lock_storage = [&](bool schema_synced) -> std::tuple { + auto get_and_lock_storage = [&](bool schema_synced, TableID table_id) -> std::tuple { /// Get storage in case it's dropped then re-created. // If schema synced, call getTable without try, leading to exception on table not existing. - auto managed_storage = tmt.getStorages().get(table_id); - if (!managed_storage) + auto table_store = tmt.getStorages().get(table_id); + if (!table_store) { if (schema_synced) throw TiFlashException("Table " + std::to_string(table_id) + " doesn't exist.", Errors::Table::NotExists); else - return {nullptr, TableStructureLockHolder{}, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, false}; + return {{}, {}, {}, false}; } - if (managed_storage->engineType() != ::TiDB::StorageEngine::TMT && managed_storage->engineType() != ::TiDB::StorageEngine::DT) + if (table_store->engineType() != ::TiDB::StorageEngine::TMT && table_store->engineType() != ::TiDB::StorageEngine::DT) { - throw TiFlashException("Specifying schema_version for non-managed storage: " + managed_storage->getName() - + ", table: " + managed_storage->getTableName() + ", id: " + DB::toString(table_id) + " is not allowed", + throw TiFlashException("Specifying schema_version for non-managed storage: " + table_store->getName() + + ", table: " + table_store->getTableName() + ", id: " + DB::toString(table_id) + " is not allowed", Errors::Coprocessor::Internal); } - auto lock = managed_storage->lockStructureForShare(context.getCurrentQueryId()); + auto lock = table_store->lockStructureForShare(context.getCurrentQueryId()); /// Check schema version, requiring TiDB/TiSpark and TiFlash both use exactly the same schema. // We have three schema versions, two in TiFlash: @@ -403,7 +485,7 @@ std::tuple DAGStorageInterpreter // 2. Global: the version that TiFlash global schema is at. // And one from TiDB/TiSpark: // 3. Query: the version that TiDB/TiSpark used for this query. - auto storage_schema_version = managed_storage->getTableInfo().schema_version; + auto storage_schema_version = table_store->getTableInfo().schema_version; // Not allow storage > query in any case, one example is time travel queries. if (storage_schema_version > query_schema_version) throw TiFlashException("Table " + std::to_string(table_id) + " schema version " + std::to_string(storage_schema_version) @@ -412,28 +494,59 @@ std::tuple DAGStorageInterpreter // From now on we have storage <= query. // If schema was synced, it implies that global >= query, as mentioned above we have storage <= query, we are OK to serve. if (schema_synced) - return {managed_storage, lock, storage_schema_version, true}; + return {table_store, lock, storage_schema_version, true}; // From now on the schema was not synced. // 1. storage == query, TiDB/TiSpark is using exactly the same schema that altered this table, we are just OK to serve. // 2. global >= query, TiDB/TiSpark is using a schema older than TiFlash global, but as mentioned above we have storage <= query, // meaning that the query schema is still newer than the time when this table was last altered, so we still OK to serve. if (storage_schema_version == query_schema_version || global_schema_version >= query_schema_version) - return {managed_storage, lock, storage_schema_version, true}; + return {table_store, lock, storage_schema_version, true}; // From now on we have global < query. // Return false for outer to sync and retry. - return {nullptr, TableStructureLockHolder{}, storage_schema_version, false}; + return {nullptr, {}, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, false}; }; - auto log_schema_version = [&](const String & result, Int64 storage_schema_version) { - LOG_FMT_INFO( - log, - "{} Table {} schema {} Schema version [storage, global, query]: [{}, {}, {}].", - __PRETTY_FUNCTION__, - table_id, - result, - storage_schema_version, - global_schema_version, - query_schema_version); + auto get_and_lock_storages = [&](bool schema_synced) -> std::tuple, std::vector, std::vector, bool> { + std::vector table_storages; + std::vector table_locks; + std::vector table_schema_versions; + auto [logical_table_storage, logical_table_lock, logical_table_storage_schema_version, ok] = get_and_lock_storage(schema_synced, logical_table_id); + if (!ok) + return {{}, {}, {}, false}; + table_storages.emplace_back(std::move(logical_table_storage)); + table_locks.emplace_back(std::move(logical_table_lock)); + table_schema_versions.push_back(logical_table_storage_schema_version); + if (!table_scan.isPartitionTableScan()) + { + return {table_storages, table_locks, table_schema_versions, true}; + } + for (auto const physical_table_id : table_scan.getPhysicalTableIDs()) + { + auto [physical_table_storage, physical_table_lock, physical_table_storage_schema_version, ok] = get_and_lock_storage(schema_synced, physical_table_id); + if (!ok) + { + return {{}, {}, {}, false}; + } + table_storages.emplace_back(std::move(physical_table_storage)); + table_locks.emplace_back(std::move(physical_table_lock)); + table_schema_versions.push_back(physical_table_storage_schema_version); + } + return {table_storages, table_locks, table_schema_versions, true}; + }; + + auto log_schema_version = [&](const String & result, const std::vector & storage_schema_versions) { + FmtBuffer buffer; + buffer.fmtAppend("{} Table {} schema {} Schema version [storage, global, query]: [{}, {}, {}]", __PRETTY_FUNCTION__, logical_table_id, result, storage_schema_versions[0], global_schema_version, query_schema_version); + if (table_scan.isPartitionTableScan()) + { + assert(storage_schema_versions.size() == 1 + table_scan.getPhysicalTableIDs().size()); + for (size_t i = 0; i < table_scan.getPhysicalTableIDs().size(); ++i) + { + const auto physical_table_id = table_scan.getPhysicalTableIDs()[i]; + buffer.fmtAppend(", Table {} schema {} Schema version [storage, global, query]: [{}, {}, {}]", physical_table_id, result, storage_schema_versions[1 + i], global_schema_version, query_schema_version); + } + } + return buffer.toString(); }; auto sync_schema = [&] { @@ -441,77 +554,81 @@ std::tuple DAGStorageInterpreter GET_METRIC(tiflash_schema_trigger_count, type_cop_read).Increment(); tmt.getSchemaSyncer()->syncSchemas(context); auto schema_sync_cost = std::chrono::duration_cast(Clock::now() - start_time).count(); - LOG_FMT_INFO(log, "{} Table {} schema sync cost {}ms.", __PRETTY_FUNCTION__, table_id, schema_sync_cost); + LOG_FMT_INFO(log, "{} Table {} schema sync cost {}ms.", __PRETTY_FUNCTION__, logical_table_id, schema_sync_cost); }; /// Try get storage and lock once. - auto [storage_, lock, storage_schema_version, ok] = get_and_lock_storage(false); + auto [storages, locks, storage_schema_versions, ok] = get_and_lock_storages(false); if (ok) { - log_schema_version("OK, no syncing required.", storage_schema_version); - return {storage_, lock}; + LOG_FMT_INFO(log, "{}", log_schema_version("OK, no syncing required.", storage_schema_versions)); } - + else /// If first try failed, sync schema and try again. { - log_schema_version("not OK, syncing schemas.", storage_schema_version); + LOG_FMT_INFO(log, "not OK, syncing schemas."); sync_schema(); - std::tie(storage_, lock, storage_schema_version, ok) = get_and_lock_storage(true); + std::tie(storages, locks, storage_schema_versions, ok) = get_and_lock_storages(true); if (ok) { - log_schema_version("OK after syncing.", storage_schema_version); - return {storage_, lock}; + LOG_FMT_INFO(log, "{}", log_schema_version("OK after syncing.", storage_schema_versions)); } - - throw TiFlashException("Shouldn't reach here", Errors::Coprocessor::Internal); + else + throw TiFlashException("Shouldn't reach here", Errors::Coprocessor::Internal); } + for (size_t i = 0; i < storages.size(); ++i) + { + auto const table_id = storages[i]->getTableInfo().id; + storages_with_lock[table_id] = {std::move(storages[i]), std::move(locks[i])}; + } + return storages_with_lock; } -std::tuple, String> DAGStorageInterpreter::getColumnsForTableScan(Int64 max_columns_to_read) +std::tuple> DAGStorageInterpreter::getColumnsForTableScan(Int64 max_columns_to_read) { // todo handle alias column - if (max_columns_to_read && table_scan.columns().size() > max_columns_to_read) + if (max_columns_to_read && table_scan.getColumnSize() > max_columns_to_read) { throw TiFlashException("Limit for number of columns to read exceeded. " "Requested: " - + toString(table_scan.columns().size()) + ", maximum: " + toString(max_columns_to_read), + + toString(table_scan.getColumnSize()) + ", maximum: " + toString(max_columns_to_read), Errors::BroadcastJoin::TooManyColumns); } - Names table_scan_required_columns; - NamesAndTypes table_scan_source_columns; + Names required_columns_tmp; + NamesAndTypes source_columns_tmp; std::vector need_cast_column; - need_cast_column.reserve(table_scan.columns_size()); - String table_scan_handle_column_name = MutableSupport::tidb_pk_column_name; - if (auto pk_handle_col = storage->getTableInfo().getPKHandleColumn()) - table_scan_handle_column_name = pk_handle_col->get().name; + need_cast_column.reserve(table_scan.getColumnSize()); + String handle_column_name = MutableSupport::tidb_pk_column_name; + if (auto pk_handle_col = storage_for_logical_table->getTableInfo().getPKHandleColumn()) + handle_column_name = pk_handle_col->get().name; - for (Int32 i = 0; i < table_scan.columns_size(); i++) + for (Int32 i = 0; i < table_scan.getColumnSize(); ++i) { - auto const & ci = table_scan.columns(i); + auto const & ci = table_scan.getColumns()[i]; ColumnID cid = ci.column_id(); - // Column ID -1 return the handle column, -3 return the extra physical table id column + // Column ID -1 return the handle column String name; if (cid == TiDBPkColumnID) - name = table_scan_handle_column_name; + name = handle_column_name; else if (cid == ExtraTableIDColumnID) name = MutableSupport::extra_table_id_column_name; else - name = storage->getTableInfo().getColumnName(cid); + name = storage_for_logical_table->getTableInfo().getColumnName(cid); if (cid == ExtraTableIDColumnID) { NameAndTypePair extra_table_id_column_pair = {name, MutableSupport::extra_table_id_column_type}; - table_scan_source_columns.emplace_back(std::move(extra_table_id_column_pair)); + source_columns_tmp.emplace_back(std::move(extra_table_id_column_pair)); } else { - auto pair = storage->getColumns().getPhysical(name); - table_scan_source_columns.emplace_back(std::move(pair)); + auto pair = storage_for_logical_table->getColumns().getPhysical(name); + source_columns_tmp.emplace_back(std::move(pair)); } - table_scan_required_columns.emplace_back(std::move(name)); + required_columns_tmp.emplace_back(std::move(name)); if (cid != -1 && ci.tp() == TiDB::TypeTimestamp) need_cast_column.push_back(ExtraCastAfterTSMode::AppendTimeZoneCast); else if (cid != -1 && ci.tp() == TiDB::TypeTime) @@ -520,86 +637,56 @@ std::tuple, String> DAGS need_cast_column.push_back(ExtraCastAfterTSMode::None); } - return {table_scan_required_columns, table_scan_source_columns, need_cast_column, table_scan_handle_column_name}; + return {required_columns_tmp, source_columns_tmp, need_cast_column}; } -std::tuple, std::optional> DAGStorageInterpreter::buildRemoteTS() +void DAGStorageInterpreter::buildRemoteRequests() { - const DAGContext & dag_context = *context.getDAGContext(); - if (region_retry.empty()) - return std::make_tuple(std::nullopt, std::nullopt); - - for (const auto & r : region_retry) + std::unordered_map region_id_to_table_id_map; + std::unordered_map retry_regions_map; + for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) { - context.getQueryContext().getDAGContext()->retry_regions.push_back(r.get()); + const auto & table_regions_info = context.getDAGContext()->getTableRegionsInfoByTableID(physical_table_id); + for (const auto & e : table_regions_info.local_regions) + region_id_to_table_id_map[e.first] = physical_table_id; + for (const auto & r : table_regions_info.remote_regions) + retry_regions_map[physical_table_id].emplace_back(std::cref(r)); } - auto print_retry_regions = [this] { - FmtBuffer buffer; - buffer.fmtAppend("Start to retry {} regions (", region_retry.size()); - buffer.joinStr( - region_retry.cbegin(), - region_retry.cend(), - [](const auto & r, FmtBuffer & fb) { fb.fmtAppend("{}", r.get().region_id); }, - ","); - buffer.append(")"); - return buffer.toString(); - }; - LOG_INFO(log, print_retry_regions()); - - DAGSchema schema; - tipb::DAGRequest dag_req; - auto * executor = dag_req.mutable_root_executor(); - if (query_block.selection != nullptr) + for (auto & r : region_retry_from_local_region) { - executor->set_tp(tipb::ExecType::TypeSelection); - executor->set_executor_id(query_block.selection->executor_id()); - auto * selection = executor->mutable_selection(); - for (const auto & condition : query_block.selection->selection().conditions()) - *selection->add_conditions() = condition; - executor = selection->mutable_child(); + retry_regions_map[region_id_to_table_id_map[r.get().region_id]].emplace_back(r); } + + for (const auto physical_table_id : table_scan.getPhysicalTableIDs()) { - const auto & table_info = storage->getTableInfo(); - tipb::Executor * ts_exec = executor; - ts_exec->set_tp(tipb::ExecType::TypeTableScan); - ts_exec->set_executor_id(query_block.source->executor_id()); - *(ts_exec->mutable_tbl_scan()) = table_scan; + const auto & retry_regions = retry_regions_map[physical_table_id]; + if (retry_regions.empty()) + continue; - for (int i = 0; i < table_scan.columns().size(); ++i) - { - const auto & col = table_scan.columns(i); - auto col_id = col.column_id(); + for (const auto & r : retry_regions) + context.getDAGContext()->retry_regions.push_back(r.get()); - if (col_id == DB::TiDBPkColumnID) - { - ColumnInfo ci; - ci.tp = TiDB::TypeLongLong; - ci.setPriKeyFlag(); - ci.setNotNullFlag(); - schema.emplace_back(std::make_pair(handle_column_name, std::move(ci))); - } - else - { - const auto & col_info = table_info.getColumnInfo(col_id); - schema.emplace_back(std::make_pair(col_info.name, col_info)); - } - dag_req.add_output_offsets(i); - } - dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); - dag_req.set_force_encode_type(true); + remote_requests.push_back(RemoteRequest::build( + retry_regions, + *context.getDAGContext(), + table_scan, + storages_with_structure_lock[physical_table_id].storage->getTableInfo(), + query_block.selection, + log)); } - /// do not collect execution summaries because in this case because the execution summaries - /// will be collected by CoprocessorBlockInputStream - dag_req.set_collect_execution_summaries(false); - const auto & original_dag_req = *dag_context.dag_request; - if (original_dag_req.has_time_zone_name() && !original_dag_req.time_zone_name().empty()) - dag_req.set_time_zone_name(original_dag_req.time_zone_name()); - if (original_dag_req.has_time_zone_offset()) - dag_req.set_time_zone_offset(original_dag_req.time_zone_offset()); - return std::make_tuple(dag_req, schema); } +void DAGStorageInterpreter::releaseAlterLocks() +{ + // The DeltaTree engine ensures that once input streams are created, the caller can get a consistent result + // from those streams even if DDL operations are applied. Release the alter lock so that reading does not + // block DDL operations, keep the drop lock so that the storage not to be dropped during reading. + for (auto storage_with_lock : storages_with_structure_lock) + { + drop_locks.emplace_back(std::get<1>(std::move(storage_with_lock.second.lock).release())); + } +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index 41d9874ecaa..384c6ce6862 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -5,13 +5,17 @@ #include #include #include +#include +#include #include #include +#include #include #include #include #include #include +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" @@ -23,8 +27,7 @@ namespace DB { -using RegionRetryList = std::list>; - +using TablesRegionInfoMap = std::unordered_map>; /// DAGStorageInterpreter encapsulates operations around storage during interprete stage. /// It's only intended to be used by DAGQueryBlockInterpreter. /// After DAGStorageInterpreter::execute some of its members will be transfered to DAGQueryBlockInterpreter. @@ -34,7 +37,7 @@ class DAGStorageInterpreter DAGStorageInterpreter( Context & context_, const DAGQueryBlock & query_block_, - const tipb::TableScan & ts, + const TiDBTableScan & table_scan, const std::vector & conditions_, size_t max_streams_); @@ -47,42 +50,46 @@ class DAGStorageInterpreter std::unique_ptr analyzer; std::vector is_need_add_cast_column; - /// it should be hash map because duplicated region id may occur if merge regions to retry of dag. - RegionRetryList region_retry; - /// Hold read lock on both `alter_lock` and `drop_lock` until the local input streams are created. - /// We need an immuntable structure to build the TableScan operator and create snapshot input streams - /// of storage. After the input streams created, the `alter_lock` can be released so that reading - /// won't block DDL operations. - TableStructureLockHolder table_structure_lock; - std::optional dag_request; - std::optional dag_schema; + /// it shouldn't be hash map because duplicated region id may occur if merge regions to retry of dag. + RegionRetryList region_retry_from_local_region; + TableLockHolders drop_locks; + std::vector remote_requests; BlockInputStreamPtr null_stream_if_empty; private: + struct StorageWithStructureLock + { + ManageableStoragePtr storage; + TableStructureLockHolder lock; + }; LearnerReadSnapshot doCopLearnerRead(); LearnerReadSnapshot doBatchCopLearnerRead(); void doLocalRead(DAGPipeline & pipeline, size_t max_block_size); - std::tuple getAndLockStorage(Int64 query_schema_version); + std::unordered_map getAndLockStorages(Int64 query_schema_version); + + std::tuple> getColumnsForTableScan(Int64 max_columns_to_read); - std::tuple, String> getColumnsForTableScan(Int64 max_columns_to_read); + void buildRemoteRequests(); - std::tuple, std::optional> buildRemoteTS(); + void releaseAlterLocks(); + + std::unordered_map generateSelectQueryInfos(); /// passed from caller, doesn't change during DAGStorageInterpreter's lifetime Context & context; const DAGQueryBlock & query_block; - const tipb::TableScan & table_scan; + const TiDBTableScan & table_scan; const std::vector & conditions; size_t max_streams; LogWithPrefixPtr log; /// derived from other members, doesn't change during DAGStorageInterpreter's lifetime - TableID table_id; + TableID logical_table_id; const Settings & settings; TMTContext & tmt; @@ -92,10 +99,14 @@ class DAGStorageInterpreter // We need to validate regions snapshot after getting streams from storage. LearnerReadSnapshot learner_read_snapshot; /// Table from where to read data, if not subquery. - ManageableStoragePtr storage; + /// Hold read lock on both `alter_lock` and `drop_lock` until the local input streams are created. + /// We need an immutable structure to build the TableScan operator and create snapshot input streams + /// of storage. After the input streams created, the `alter_lock` can be released so that reading + /// won't block DDL operations. + std::unordered_map storages_with_structure_lock; + ManageableStoragePtr storage_for_logical_table; Names required_columns; NamesAndTypes source_columns; - String handle_column_name; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index bfbe34aae58..3d1f6643590 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -560,7 +560,7 @@ const std::unordered_map scalar_func_map({ //{tipb::ScalarFuncSig::Timestamp2Args, "cast"}, //{tipb::ScalarFuncSig::TimestampLiteral, "cast"}, - //{tipb::ScalarFuncSig::LastDay, "cast"}, + {tipb::ScalarFuncSig::LastDay, "tidbLastDay"}, {tipb::ScalarFuncSig::StrToDateDate, "strToDateDate"}, {tipb::ScalarFuncSig::StrToDateDatetime, "strToDateDatetime"}, // {tipb::ScalarFuncSig::StrToDateDuration, "cast"}, @@ -1153,7 +1153,7 @@ SortDescription getSortDescription(const std::vector & order_co // MySQL/TiDB treats NULL as "minimum". int nulls_direction = -1; TiDB::TiDBCollatorPtr collator = nullptr; - if (removeNullable(order_columns[i].type)->isString()) + if (order_columns[i].type->isString()) collator = getCollatorFromExpr(by_items[i].expr()); order_descr.emplace_back(name, direction, nulls_direction, collator); diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp new file mode 100644 index 00000000000..8514e3fdff2 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp @@ -0,0 +1,92 @@ +#include +#include +#include + +namespace DB +{ +RemoteRequest RemoteRequest::build(const RegionRetryList & retry_regions, DAGContext & dag_context, const TiDBTableScan & table_scan, const TiDB::TableInfo & table_info, const tipb::Executor * selection, LogWithPrefixPtr & log) +{ + auto print_retry_regions = [&retry_regions, &table_info] { + FmtBuffer buffer; + buffer.fmtAppend("Start to build remote request for {} regions (", retry_regions.size()); + buffer.joinStr( + retry_regions.cbegin(), + retry_regions.cend(), + [](const auto & r, FmtBuffer & fb) { fb.fmtAppend("{}", r.get().region_id); }, + ","); + buffer.fmtAppend(") for table {}", table_info.id); + return buffer.toString(); + }; + LOG_FMT_INFO(log, "{}", print_retry_regions()); + + DAGSchema schema; + tipb::DAGRequest dag_req; + auto * executor = dag_req.mutable_root_executor(); + if (selection != nullptr) + { + executor->set_tp(tipb::ExecType::TypeSelection); + executor->set_executor_id(selection->executor_id()); + auto * new_selection = executor->mutable_selection(); + for (const auto & condition : selection->selection().conditions()) + *new_selection->add_conditions() = condition; + executor = new_selection->mutable_child(); + } + + { + tipb::Executor * ts_exec = executor; + ts_exec->set_tp(tipb::ExecType::TypeTableScan); + ts_exec->set_executor_id(table_scan.getTableScanExecutorID()); + auto * mutable_table_scan = ts_exec->mutable_tbl_scan(); + table_scan.constructTableScanForRemoteRead(mutable_table_scan, table_info.id); + + String handle_column_name = MutableSupport::tidb_pk_column_name; + if (auto pk_handle_col = table_info.getPKHandleColumn()) + handle_column_name = pk_handle_col->get().name; + + for (int i = 0; i < table_scan.getColumnSize(); ++i) + { + const auto & col = table_scan.getColumns()[i]; + auto col_id = col.column_id(); + + if (col_id == DB::TiDBPkColumnID) + { + ColumnInfo ci; + ci.tp = TiDB::TypeLongLong; + ci.setPriKeyFlag(); + ci.setNotNullFlag(); + schema.emplace_back(std::make_pair(handle_column_name, std::move(ci))); + } + else if (col_id == ExtraTableIDColumnID) + { + ColumnInfo ci; + ci.tp = TiDB::TypeLongLong; + schema.emplace_back(std::make_pair(MutableSupport::extra_table_id_column_name, std::move(ci))); + } + else + { + const auto & col_info = table_info.getColumnInfo(col_id); + schema.emplace_back(std::make_pair(col_info.name, col_info)); + } + dag_req.add_output_offsets(i); + } + dag_req.set_encode_type(tipb::EncodeType::TypeCHBlock); + dag_req.set_force_encode_type(true); + } + /// do not collect execution summaries because in this case because the execution summaries + /// will be collected by CoprocessorBlockInputStream + dag_req.set_collect_execution_summaries(false); + const auto & original_dag_req = *dag_context.dag_request; + if (original_dag_req.has_time_zone_name() && !original_dag_req.time_zone_name().empty()) + dag_req.set_time_zone_name(original_dag_req.time_zone_name()); + if (original_dag_req.has_time_zone_offset()) + dag_req.set_time_zone_offset(original_dag_req.time_zone_offset()); + std::vector key_ranges; + for (const auto & region : retry_regions) + { + for (const auto & range : region.get().key_ranges) + key_ranges.emplace_back(*range.first, *range.second); + } + sort(key_ranges.begin(), key_ranges.end()); + return {std::move(dag_req), std::move(schema), std::move(key_ranges)}; +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.h b/dbms/src/Flash/Coprocessor/RemoteRequest.h new file mode 100644 index 00000000000..10a7a415af7 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop + +#include + +namespace DB +{ +using RegionRetryList = std::list>; +using DAGColumnInfo = std::pair; +using DAGSchema = std::vector; + +struct RemoteRequest +{ + RemoteRequest(tipb::DAGRequest && dag_request_, DAGSchema && schema_, std::vector && key_ranges_) + : dag_request(std::move(dag_request_)) + , schema(std::move(schema_)) + , key_ranges(std::move(key_ranges_)) + {} + tipb::DAGRequest dag_request; + DAGSchema schema; + /// the sorted key ranges + std::vector key_ranges; + static RemoteRequest build(const RegionRetryList & retry_regions, DAGContext & dag_context, const TiDBTableScan & table_scan, const TiDB::TableInfo & table_info, const tipb::Executor * selection, LogWithPrefixPtr & log); +}; +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp new file mode 100644 index 00000000000..297154f08ce --- /dev/null +++ b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp @@ -0,0 +1,97 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace FailPoints +{ +extern const char force_no_local_region_for_mpp_task[]; +} // namespace FailPoints + +SingleTableRegions & TablesRegionsInfo::getOrCreateTableRegionInfoByTableID(Int64 table_id) +{ + if (is_single_table) + return table_regions_info_map.begin()->second; + if (table_regions_info_map.find(table_id) == table_regions_info_map.end()) + { + table_regions_info_map[table_id] = SingleTableRegions(); + } + return table_regions_info_map.find(table_id)->second; +} +const SingleTableRegions & TablesRegionsInfo::getTableRegionInfoByTableID(Int64 table_id) const +{ + if (is_single_table) + return table_regions_info_map.begin()->second; + if (table_regions_info_map.find(table_id) != table_regions_info_map.end()) + return table_regions_info_map.find(table_id)->second; + throw TiFlashException(fmt::format("Can't find region info for table id: {}", table_id), Errors::Coprocessor::BadRequest); +} + +static bool needRemoteRead(const RegionInfo & region_info, const TMTContext & tmt_context) +{ + fiu_do_on(FailPoints::force_no_local_region_for_mpp_task, { return true; }); + RegionPtr current_region = tmt_context.getKVStore()->getRegion(region_info.region_id); + if (current_region == nullptr || current_region->peerState() != raft_serverpb::PeerState::Normal) + return true; + auto meta_snap = current_region->dumpRegionMetaSnapshot(); + return meta_snap.ver != region_info.region_version; +} + +static void insertRegionInfoToTablesRegionInfo(const google::protobuf::RepeatedPtrField & regions, Int64 table_id, TablesRegionsInfo & tables_region_infos, std::unordered_set & local_region_id_set, const TMTContext & tmt_context) +{ + auto & table_region_info = tables_region_infos.getOrCreateTableRegionInfoByTableID(table_id); + for (const auto & r : regions) + { + RegionInfo region_info(r.region_id(), r.region_epoch().version(), r.region_epoch().conf_ver(), CoprocessorHandler::GenCopKeyRange(r.ranges()), nullptr); + if (region_info.key_ranges.empty()) + { + throw TiFlashException( + fmt::format("Income key ranges is empty for region: {}", region_info.region_id), + Errors::Coprocessor::BadRequest); + } + /// TiFlash does not support regions with duplicated region id, so for regions with duplicated + /// region id, only the first region will be treated as local region + /// + /// 1. Currently TiDB can't provide a consistent snapshot of the region cache and it may be updated during the + /// planning stage of a query. The planner may see multiple versions of one region (on one TiFlash node). + /// 2. Two regions with same region id won't have overlapping key ranges. + /// 3. TiFlash will pick the right version of region for local read and others for remote read. + /// 4. The remote read will fetch the newest region info via key ranges. So it is possible to find the region + /// is served by the same node (but still read from remote). + bool duplicated_region = local_region_id_set.count(region_info.region_id) > 0; + + if (duplicated_region || needRemoteRead(region_info, tmt_context)) + table_region_info.remote_regions.push_back(region_info); + else + { + table_region_info.local_regions.insert(std::make_pair(region_info.region_id, region_info)); + local_region_id_set.emplace(region_info.region_id); + } + } +} + +TablesRegionsInfo TablesRegionsInfo::create( + const google::protobuf::RepeatedPtrField & regions, + const google::protobuf::RepeatedPtrField & table_regions, + const TMTContext & tmt_context) +{ + assert(regions.empty() || table_regions.empty()); + TablesRegionsInfo tables_regions_info(!regions.empty()); + std::unordered_set local_region_id_set; + if (!regions.empty()) + insertRegionInfoToTablesRegionInfo(regions, InvalidTableID, tables_regions_info, local_region_id_set, tmt_context); + else + { + for (const auto & table_region : table_regions) + { + assert(table_region.physical_table_id() != InvalidTableID); + insertRegionInfoToTablesRegionInfo(table_region.regions(), table_region.physical_table_id(), tables_regions_info, local_region_id_set, tmt_context); + } + assert(static_cast(table_regions.size()) == tables_regions_info.tableCount()); + } + return tables_regions_info; +} + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h new file mode 100644 index 00000000000..cc921163ca0 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h @@ -0,0 +1,76 @@ +#pragma once +#include +#include +#include + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include +#include +#pragma GCC diagnostic pop + +namespace DB +{ +struct SingleTableRegions +{ + RegionInfoMap local_regions; + RegionInfoList remote_regions; + size_t regionCount() const { return local_regions.size() + remote_regions.size(); } +}; + +/// this class contains all the region info for all physical table scans in a dag request, +/// currently, one dag request only contain one logical table scan, if the target table +/// is non-partition table, then is_single_table is true, otherwise, is_single_table is false +/// since one partition table may contain multiple physical tables +class TablesRegionsInfo +{ +public: + static TablesRegionsInfo create( + const google::protobuf::RepeatedPtrField & regions, + const google::protobuf::RepeatedPtrField & table_regions, + const TMTContext & tmt_context); + TablesRegionsInfo() + : is_single_table(false) + {} + explicit TablesRegionsInfo(bool is_single_table_) + : is_single_table(is_single_table_) + { + if (is_single_table) + table_regions_info_map[InvalidTableID] = SingleTableRegions(); + } + SingleTableRegions & getSingleTableRegions() + { + assert(is_single_table); + return table_regions_info_map.begin()->second; + } + SingleTableRegions & getOrCreateTableRegionInfoByTableID(TableID table_id); + const SingleTableRegions & getTableRegionInfoByTableID(TableID table_id) const; + const std::unordered_map & getTableRegionsInfoMap() const + { + return table_regions_info_map; + } + bool containsRegionsInfoForTable(TableID table_id) const + { + /// for single table, skip check the table_id since we use use InvalidTableID as a place holder + if (is_single_table) + return true; + return table_regions_info_map.find(table_id) != table_regions_info_map.end(); + } + UInt64 regionCount() const + { + UInt64 ret = 0; + for (const auto & entry : table_regions_info_map) + ret += entry.second.regionCount(); + return ret; + } + UInt64 tableCount() const + { + return table_regions_info_map.size(); + } + +private: + bool is_single_table; + std::unordered_map table_regions_info_map; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp new file mode 100644 index 00000000000..c2078c096b0 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.cpp @@ -0,0 +1,62 @@ +#include + +namespace DB +{ +TiDBTableScan::TiDBTableScan(const tipb::Executor * table_scan_, const DAGContext & dag_context) + : table_scan(table_scan_) + , is_partition_table_scan(table_scan->tp() == tipb::TypePartitionTableScan) + , columns(is_partition_table_scan ? table_scan->partition_table_scan().columns() : table_scan->tbl_scan().columns()) +{ + if (is_partition_table_scan) + { + if (table_scan->partition_table_scan().has_table_id()) + logical_table_id = table_scan->partition_table_scan().table_id(); + else + throw TiFlashException("Partition table scan without table id.", Errors::Coprocessor::BadRequest); + std::set all_physical_table_ids; + for (const auto & partition_table_id : table_scan->partition_table_scan().partition_ids()) + { + if (all_physical_table_ids.count(partition_table_id) > 0) + throw TiFlashException("Partition table scan contains duplicated physical table ids.", Errors::Coprocessor::BadRequest); + all_physical_table_ids.insert(partition_table_id); + if (dag_context.containsRegionsInfoForTable(partition_table_id)) + physical_table_ids.push_back(partition_table_id); + } + std::sort(physical_table_ids.begin(), physical_table_ids.end()); + if (physical_table_ids.size() != dag_context.tables_regions_info.tableCount()) + throw TiFlashException("Partition table scan contains table_region_info that is not belongs to the partition table.", Errors::Coprocessor::BadRequest); + } + else + { + if (table_scan->tbl_scan().next_read_engine() != tipb::EngineType::Local) + throw TiFlashException("Unsupported remote query.", Errors::Coprocessor::BadRequest); + + if (table_scan->tbl_scan().has_table_id()) + logical_table_id = table_scan->tbl_scan().table_id(); + else + throw TiFlashException("table scan without table id.", Errors::Coprocessor::BadRequest); + physical_table_ids.push_back(logical_table_id); + } +} +void TiDBTableScan::constructTableScanForRemoteRead(tipb::TableScan * tipb_table_scan, TableID table_id) const +{ + if (is_partition_table_scan) + { + const auto & partition_table_scan = table_scan->partition_table_scan(); + tipb_table_scan->set_table_id(table_id); + for (const auto & column : partition_table_scan.columns()) + *tipb_table_scan->add_columns() = column; + tipb_table_scan->set_desc(partition_table_scan.desc()); + for (auto id : partition_table_scan.primary_column_ids()) + tipb_table_scan->add_primary_column_ids(id); + tipb_table_scan->set_next_read_engine(tipb::EngineType::Local); + for (auto id : partition_table_scan.primary_prefix_column_ids()) + tipb_table_scan->add_primary_prefix_column_ids(id); + } + else + { + *tipb_table_scan = table_scan->tbl_scan(); + tipb_table_scan->set_table_id(table_id); + } +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/TiDBTableScan.h b/dbms/src/Flash/Coprocessor/TiDBTableScan.h new file mode 100644 index 00000000000..5a1ed432683 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/TiDBTableScan.h @@ -0,0 +1,55 @@ +#pragma once + +#include + +#include + +namespace DB +{ +/// TiDBTableScan is a wrap to hide the difference of `TableScan` and `PartitionTableScan` +class TiDBTableScan +{ +public: + TiDBTableScan(const tipb::Executor * table_scan_, const DAGContext & dag_context); + bool isPartitionTableScan() const + { + return is_partition_table_scan; + } + Int64 getColumnSize() const + { + return columns.size(); + } + const google::protobuf::RepeatedPtrField & getColumns() const + { + return columns; + } + void constructTableScanForRemoteRead(tipb::TableScan * tipb_table_scan, TableID table_id) const; + Int64 getLogicalTableID() const + { + return logical_table_id; + } + const std::vector & getPhysicalTableIDs() const + { + return physical_table_ids; + } + String getTableScanExecutorID() const + { + return table_scan->executor_id(); + } + +private: + const tipb::Executor * table_scan; + bool is_partition_table_scan; + const google::protobuf::RepeatedPtrField & columns; + /// logical_table_id is the table id for a TiDB' table, while if the + /// TiDB table is partition, each partition is a physical table, and + /// the partition's table id is the physical table id. + /// So, for non-partition table, physical_table_ids.size() == 1, and + /// physical_table_ids[0] == logical_table_id, + /// for partition table, logical_table_id is the partition table id, + /// physical_table_ids contains the table ids of its partitions + std::vector physical_table_ids; + Int64 logical_table_id; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp b/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp index 6138d2183d9..bdfbe5ccddf 100644 --- a/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp +++ b/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp @@ -51,7 +51,8 @@ bool collectForReceiver(std::vector & output_field_types, const return false; } -bool collectForTableScan(std::vector & output_field_types, const tipb::TableScan & tbl_scan) +template +bool collectForTableScan(std::vector & output_field_types, const TableScanType & tbl_scan) { for (const auto & ci : tbl_scan.columns()) { @@ -133,6 +134,8 @@ bool collectForExecutor(std::vector & output_field_types, const return collectForReceiver(output_field_types, executor.exchange_receiver()); case tipb::ExecType::TypeTableScan: return collectForTableScan(output_field_types, executor.tbl_scan()); + case tipb::ExecType::TypePartitionTableScan: + return collectForTableScan(output_field_types, executor.partition_table_scan()); case tipb::ExecType::TypeJoin: return collectForJoin(output_field_types, executor); default: @@ -146,4 +149,4 @@ std::vector collectOutputFieldTypes(const tipb::DAGRequest & da traverseExecutors(&dag_request, [&output_field_types](const tipb::Executor & e) { return collectForExecutor(output_field_types, e); }); return output_field_types; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp index 8b34f93eef4..561234f577a 100644 --- a/dbms/src/Flash/CoprocessorHandler.cpp +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -42,7 +42,7 @@ std::vector> CoprocessorHandler: const ::google::protobuf::RepeatedPtrField<::coprocessor::KeyRange> & ranges) { std::vector> key_ranges; - for (auto & range : ranges) + for (const auto & range : ranges) { DecodedTiKVKeyPtr start = std::make_shared(std::string(range.start())); DecodedTiKVKeyPtr end = std::make_shared(std::string(range.end())); @@ -73,16 +73,23 @@ grpc::Status CoprocessorHandler::execute() "DAG request with rpn expression is not supported in TiFlash", Errors::Coprocessor::Unimplemented); tipb::SelectResponse dag_response; - RegionInfoMap regions; + TablesRegionsInfo tables_regions_info(true); + auto & table_regions_info = tables_regions_info.getSingleTableRegions(); const std::unordered_set bypass_lock_ts( cop_context.kv_context.resolved_locks().begin(), cop_context.kv_context.resolved_locks().end()); - regions.emplace(cop_context.kv_context.region_id(), - RegionInfo(cop_context.kv_context.region_id(), cop_context.kv_context.region_epoch().version(), cop_context.kv_context.region_epoch().conf_ver(), GenCopKeyRange(cop_request->ranges()), &bypass_lock_ts)); + table_regions_info.local_regions.emplace( + cop_context.kv_context.region_id(), + RegionInfo( + cop_context.kv_context.region_id(), + cop_context.kv_context.region_epoch().version(), + cop_context.kv_context.region_epoch().conf_ver(), + GenCopKeyRange(cop_request->ranges()), + &bypass_lock_ts)); DAGContext dag_context(dag_request); - dag_context.regions_for_local_read = std::move(regions); + dag_context.tables_regions_info = std::move(tables_regions_info); dag_context.log = std::make_shared(log, ""); dag_context.tidb_host = cop_context.db_context.getClientInfo().current_address.toString(); cop_context.db_context.setDAGContext(&dag_context); diff --git a/dbms/src/Flash/CoprocessorHandler.h b/dbms/src/Flash/CoprocessorHandler.h index 72c0dcf5a8a..d8a0417e47a 100644 --- a/dbms/src/Flash/CoprocessorHandler.h +++ b/dbms/src/Flash/CoprocessorHandler.h @@ -35,15 +35,15 @@ class CoprocessorHandler public: CoprocessorHandler(CoprocessorContext & cop_context_, const coprocessor::Request * cop_request_, coprocessor::Response * response_); - ~CoprocessorHandler() = default; + virtual ~CoprocessorHandler() = default; - grpc::Status execute(); + virtual grpc::Status execute(); static std::vector> GenCopKeyRange( const ::google::protobuf::RepeatedPtrField<::coprocessor::KeyRange> & ranges); protected: - grpc::Status recordError(grpc::StatusCode err_code, const String & err_msg); + virtual grpc::Status recordError(grpc::StatusCode err_code, const String & err_msg); protected: enum diff --git a/dbms/src/Flash/Mpp/MPPHandler.cpp b/dbms/src/Flash/Mpp/MPPHandler.cpp index e345e497723..58903772dfc 100644 --- a/dbms/src/Flash/Mpp/MPPHandler.cpp +++ b/dbms/src/Flash/Mpp/MPPHandler.cpp @@ -35,12 +35,15 @@ grpc::Status MPPHandler::execute(const ContextPtr & context, mpp::DispatchTaskRe task = MPPTask::newTask(task_request.meta(), context); task->prepare(task_request); - for (const auto & region : context->getDAGContext()->getRegionsForRemoteRead()) + for (const auto & table_region_info : context->getDAGContext()->tables_regions_info.getTableRegionsInfoMap()) { - auto * retry_region = response->add_retry_regions(); - retry_region->set_id(region.region_id); - retry_region->mutable_region_epoch()->set_conf_ver(region.region_conf_version); - retry_region->mutable_region_epoch()->set_version(region.region_version); + for (const auto & region : table_region_info.second.remote_regions) + { + auto * retry_region = response->add_retry_regions(); + retry_region->set_id(region.region_id); + retry_region->mutable_region_epoch()->set_conf_ver(region.region_conf_version); + retry_region->mutable_region_epoch()->set_version(region.region_version); + } } if (task->isRootMPPTask()) { diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 4649adf6646..d996e269cf2 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -123,52 +123,20 @@ void MPPTask::unregisterTask() } } -bool needRemoteRead(const RegionInfo & region_info, const TMTContext & tmt_context) -{ - fiu_do_on(FailPoints::force_no_local_region_for_mpp_task, { return true; }); - RegionPtr current_region = tmt_context.getKVStore()->getRegion(region_info.region_id); - if (current_region == nullptr || current_region->peerState() != raft_serverpb::PeerState::Normal) - return true; - auto meta_snap = current_region->dumpRegionMetaSnapshot(); - return meta_snap.ver != region_info.region_version; -} - void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) { - RegionInfoMap local_regions; - RegionInfoList remote_regions; - dag_req = getDAGRequestFromStringWithRetry(task_request.encoded_plan()); TMTContext & tmt_context = context->getTMTContext(); - /// MPP task will only use key ranges in mpp::DispatchTaskRequest::regions. The ones defined in tipb::TableScan - /// will never be used and can be removed later. - /// Each MPP task will contain at most one TableScan operator belonging to one table. For those tasks without - /// TableScan, their DispatchTaskRequests won't contain any region. - for (const auto & r : task_request.regions()) - { - RegionInfo region_info(r.region_id(), r.region_epoch().version(), r.region_epoch().conf_ver(), CoprocessorHandler::GenCopKeyRange(r.ranges()), nullptr); - if (region_info.key_ranges.empty()) - { - throw TiFlashException( - fmt::format("Income key ranges is empty for region: {}", region_info.region_id), - Errors::Coprocessor::BadRequest); - } - /// TiFlash does not support regions with duplicated region id, so for regions with duplicated - /// region id, only the first region will be treated as local region - /// - /// 1. Currently TiDB can't provide a consistent snapshot of the region cache and it may be updated during the - /// planning stage of a query. The planner may see multiple versions of one region (on one TiFlash node). - /// 2. Two regions with same region id won't have overlapping key ranges. - /// 3. TiFlash will pick the right version of region for local read and others for remote read. - /// 4. The remote read will fetch the newest region info via key ranges. So it is possible to find the region - /// is served by the same node (but still read from remote). - bool duplicated_region = local_regions.find(region_info.region_id) != local_regions.end(); - - if (duplicated_region || needRemoteRead(region_info, tmt_context)) - remote_regions.push_back(region_info); - else - local_regions.insert(std::make_pair(region_info.region_id, region_info)); - } + /// MPP task will only use key ranges in mpp::DispatchTaskRequest::regions/mpp::DispatchTaskRequest::table_regions. + /// The ones defined in tipb::TableScan will never be used and can be removed later. + TablesRegionsInfo tables_regions_info = TablesRegionsInfo::create(task_request.regions(), task_request.table_regions(), tmt_context); + LOG_FMT_DEBUG( + log, + "{}: Handling {} regions from {} physical tables in MPP task", + __PRETTY_FUNCTION__, + tables_regions_info.regionCount(), + tables_regions_info.tableCount()); + // set schema ver and start ts. auto schema_ver = task_request.schema_ver(); auto start_ts = task_request.meta().start_ts(); @@ -208,8 +176,7 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) } dag_context = std::make_unique(dag_req, task_request.meta(), is_root_mpp_task); dag_context->log = log; - dag_context->regions_for_local_read = std::move(local_regions); - dag_context->regions_for_remote_read = std::move(remote_regions); + dag_context->tables_regions_info = std::move(tables_regions_info); dag_context->tidb_host = context->getClientInfo().current_address.toString(); context->setDAGContext(dag_context.get()); diff --git a/dbms/src/Flash/Statistics/TableScanImpl.h b/dbms/src/Flash/Statistics/TableScanImpl.h index 5f1ffa1aeab..3938534ee50 100644 --- a/dbms/src/Flash/Statistics/TableScanImpl.h +++ b/dbms/src/Flash/Statistics/TableScanImpl.h @@ -25,7 +25,7 @@ struct TableScanImpl static bool isMatch(const tipb::Executor * executor) { - return executor->has_tbl_scan(); + return executor->has_tbl_scan() || executor->has_partition_table_scan(); } }; using TableScanStatisticsBase = ExecutorStatistics; diff --git a/dbms/src/Flash/Statistics/traverseExecutors.cpp b/dbms/src/Flash/Statistics/traverseExecutors.cpp index 892633c9550..b83450e6f31 100644 --- a/dbms/src/Flash/Statistics/traverseExecutors.cpp +++ b/dbms/src/Flash/Statistics/traverseExecutors.cpp @@ -8,6 +8,7 @@ Children getChildren(const tipb::Executor & executor) switch (executor.tp()) { case tipb::ExecType::TypeTableScan: + case tipb::ExecType::TypePartitionTableScan: return {}; case tipb::ExecType::TypeJoin: return {&executor.join().children(0), &executor.join().children(1)}; diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index d7bc82e5c96..e9102e3bc44 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -214,10 +214,9 @@ class FunctionTiDBIPv6NumToString : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - DataTypePtr data_type = removeNullable(arguments[0]); - if (!data_type->isString()) + if (!arguments[0]->isString()) throw Exception( - fmt::format("Illegal type {} of argument of function {}", data_type->getName(), getName()), + fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return makeNullable(std::make_shared()); @@ -617,10 +616,9 @@ class FunctionTiDBIPv6StringToNum : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - DataTypePtr data_type = removeNullable(arguments[0]); - if (!data_type->isString()) + if (!arguments[0]->isString()) throw Exception( - fmt::format("Illegal type {} of argument of function {}", data_type->getName(), getName()), + fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return makeNullable(std::make_shared()); @@ -971,10 +969,9 @@ class FunctionTiDBIPv4StringToNum : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - DataTypePtr data_type = removeNullable(arguments[0]); - if (!data_type->isString()) + if (!arguments[0]->isString()) throw Exception( - fmt::format("Illegal type {} of argument of function {}", data_type->getName(), getName()), + fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return makeNullable(std::make_shared()); diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 9b86787cb7a..cf8c5596e29 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1340,9 +1340,8 @@ class FunctionIsTrueFalse : public IFunction return; } - auto src_type = removeNullable(src.type); ColumnPtr res_col; - switch (src_type->getTypeId()) + switch (src.type->getTypeId()) { case TypeIndex::UInt8: res_col = executeVec>(src.column); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index bab240f517e..04c1fe50f83 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1449,9 +1449,9 @@ class FunctionFromUnixTime : public IFunction { if (arguments.size() != 1 && arguments.size() != 2) throw Exception("Function " + getName() + " only accept 1 or 2 arguments"); - if (!removeNullable(arguments[0].type)->isDecimal() && !arguments[0].type->onlyNull()) + if (!arguments[0].type->isDecimal() && !arguments[0].type->onlyNull()) throw Exception("First argument for function " + getName() + " must be decimal type", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() == 2 && (!removeNullable(arguments[1].type)->isString() && !arguments[1].type->onlyNull())) + if (arguments.size() == 2 && (!arguments[1].type->isString() && !arguments[1].type->onlyNull())) throw Exception("Second argument of function " + getName() + " must be string/null constant", ErrorCodes::ILLEGAL_COLUMN); for (const auto & arg : arguments) @@ -1763,10 +1763,10 @@ class FunctionStrToDate : public IFunction throw Exception(fmt::format("Function {} only accept 2 arguments", getName()), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); // TODO: Maybe FixedString? - if (!removeNullable(arguments[0].type)->isString()) + if (!arguments[0].type->isString()) throw Exception(fmt::format("First argument for function {} must be String, but get {}", getName(), arguments[0].type->getName()), ErrorCodes::ILLEGAL_COLUMN); - if (!removeNullable(arguments[1].type)->isString()) + if (!arguments[1].type->isString()) throw Exception(fmt::format("Second argument for function {} must be String, but get {}", getName(), arguments[1].type->getName()), ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/FunctionsDateTime.cpp b/dbms/src/Functions/FunctionsDateTime.cpp index dbfac0e8871..9b71a75b085 100644 --- a/dbms/src/Functions/FunctionsDateTime.cpp +++ b/dbms/src/Functions/FunctionsDateTime.cpp @@ -121,6 +121,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } // namespace DB diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index 8cd5d6e7250..1024d0070c9 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -1548,17 +1549,17 @@ class FunctionTiDBTimestampDiff : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!removeNullable(arguments[0])->isString()) + if (!arguments[0]->isString()) throw Exception( fmt::format("First argument for function {} (unit) must be String", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[1])->isMyDateOrMyDateTime() && !arguments[1]->onlyNull()) + if (!arguments[1]->isMyDateOrMyDateTime() && !arguments[1]->onlyNull()) throw Exception( fmt::format("Second argument for function {} must be MyDate or MyDateTime", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[2])->isMyDateOrMyDateTime() && !arguments[2]->onlyNull()) + if (!arguments[2]->isMyDateOrMyDateTime() && !arguments[2]->onlyNull()) throw Exception( fmt::format("Third argument for function {} must be MyDate or MyDateTime", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1935,12 +1936,12 @@ class FunctionTiDBDateDiff : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!removeNullable(arguments[0])->isMyDateOrMyDateTime()) + if (!arguments[0]->isMyDateOrMyDateTime()) throw Exception( fmt::format("First argument for function {} must be MyDate or MyDateTime", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[1])->isMyDateOrMyDateTime()) + if (!arguments[1]->isMyDateOrMyDateTime()) throw Exception( fmt::format("Second argument for function {} must be MyDate or MyDateTime", getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -3241,6 +3242,118 @@ class FunctionDateTimeToString : public IFunction const Context & context; }; +template +struct TiDBLastDayTransformerImpl +{ + static_assert(std::is_same_v); + static constexpr auto name = "tidbLastDay"; + + static void execute(const Context & context, + const ColumnVector::Container & vec_from, + typename ColumnVector::Container & vec_to, + typename ColumnVector::Container & vec_null_map) + { + for (size_t i = 0; i < vec_from.size(); ++i) + { + bool is_null = false; + MyTimeBase val(vec_from[i]); + vec_to[i] = execute(context, val, is_null); + vec_null_map[i] = is_null; + } + } + + static ToFieldType execute(const Context & context, const MyTimeBase & val, bool & is_null) + { + // TiDB also considers NO_ZERO_DATE sql_mode. But sql_mode is not handled by TiFlash for now. + if (val.month == 0 || val.day == 0) + { + context.getDAGContext()->handleInvalidTime( + fmt::format("Invalid time value: month({}) or day({}) is zero", val.month, val.day), + Errors::Types::WrongValue); + is_null = true; + return 0; + } + UInt8 last_day = getLastDay(val.year, val.month); + return MyDate(val.year, val.month, last_day).toPackedUInt(); + } +}; + +// Similar to FunctionDateOrDateTimeToSomething, but also handle nullable result and mysql sql mode. +template class Transformer, bool return_nullable> +class FunctionMyDateOrMyDateTimeToSomething : public IFunction +{ +private: + const Context & context; + +public: + using ToFieldType = typename ToDataType::FieldType; + static constexpr auto name = Transformer::name; + + explicit FunctionMyDateOrMyDateTimeToSomething(const Context & context) + : context(context) + {} + static FunctionPtr create(const Context & context) { return std::make_shared(context); }; + + 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 + { + if (!arguments[0].type->isMyDateOrMyDateTime()) + throw Exception( + fmt::format("Illegal type {} of argument of function {}. Should be a date or a date with time", arguments[0].type->getName(), getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + DataTypePtr return_type = std::make_shared(); + if constexpr (return_nullable) + return_type = makeNullable(return_type); + return return_type; + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override + { + const DataTypePtr & from_type = block.getByPosition(arguments[0]).type; + + if (from_type->isMyDateOrMyDateTime()) + { + using FromFieldType = typename DataTypeMyTimeBase::FieldType; + + const ColumnVector * col_from + = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get()); + 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(); + + if constexpr (return_nullable) + { + ColumnUInt8::MutablePtr col_null_map = ColumnUInt8::create(size, 0); + ColumnUInt8::Container & vec_null_map = col_null_map->getData(); + Transformer::execute(context, vec_from, vec_to, vec_null_map); + block.getByPosition(result).column = ColumnNullable::create(std::move(col_to), std::move(col_null_map)); + } + else + { + Transformer::execute(context, vec_from, vec_to); + block.getByPosition(result).column = std::move(col_to); + } + } + else + throw Exception( + fmt::format("Illegal type {} of argument of function {}", block.getByPosition(arguments[0]).type->getName(), getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } +}; + +static constexpr bool return_nullable = true; +static constexpr bool return_not_null = false; + using FunctionToYear = FunctionDateOrDateTimeToSomething; using FunctionToQuarter = FunctionDateOrDateTimeToSomething; using FunctionToMonth = FunctionDateOrDateTimeToSomething; @@ -3259,6 +3372,7 @@ using FunctionToStartOfFiveMinute = FunctionDateOrDateTimeToSomething; using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething; using FunctionToTime = FunctionDateOrDateTimeToSomething; +using FunctionToLastDay = FunctionMyDateOrMyDateTimeToSomething; using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething; using FunctionToRelativeQuarterNum = FunctionDateOrDateTimeToSomething; diff --git a/dbms/src/Functions/FunctionsLogical.h b/dbms/src/Functions/FunctionsLogical.h index fe900ea2c5a..1706a40627b 100644 --- a/dbms/src/Functions/FunctionsLogical.h +++ b/dbms/src/Functions/FunctionsLogical.h @@ -373,7 +373,7 @@ class FunctionAnyArityLogical : public IFunction if (!(arguments[i]->isNumber() || (special_impl_for_nulls && (arguments[i]->onlyNull() - || removeNullable(arguments[i])->isNumber())))) + || arguments[i]->isNumber())))) throw Exception( "Illegal type (" + arguments[i]->getName() + ") of " + toString(i + 1) + " argument of function " + getName(), diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp index 00beba2f280..c34128db3dd 100644 --- a/dbms/src/Functions/FunctionsString.cpp +++ b/dbms/src/Functions/FunctionsString.cpp @@ -1360,7 +1360,7 @@ class FunctionTiDBConcatWithSeparator : public IFunction { if (!arguments[arg_idx]->onlyNull()) { - const auto * arg = removeNullable(arguments[arg_idx]).get(); + const auto * arg = arguments[arg_idx].get(); if (!arg->isString()) throw Exception( fmt::format("Illegal type {} of argument {} of function {}", arg->getName(), arg_idx + 1, getName()), @@ -3374,17 +3374,17 @@ class PadImpl : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!removeNullable(arguments[0])->isString()) + if (!arguments[0]->isString()) throw Exception( fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[1])->isInteger()) + if (!arguments[1]->isInteger()) throw Exception( fmt::format("Illegal type {} of second argument of function {}", arguments[1]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[2])->isString()) + if (!arguments[2]->isString()) throw Exception( fmt::format("Illegal type {} of third argument of function {}", arguments[2]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -3442,7 +3442,7 @@ class PadImpl : public IFunction void tidbExecutePad(Block & block, const ColumnNumbers & arguments, const size_t result) const { - TypeIndex type_index = removeNullable(block.getByPosition(arguments[1]).type)->getTypeId(); + TypeIndex type_index = block.getByPosition(arguments[1]).type->getTypeId(); switch (type_index) { case TypeIndex::UInt8: @@ -3501,17 +3501,17 @@ class PadUTF8Impl : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!removeNullable(arguments[0])->isString()) + if (!arguments[0]->isString()) throw Exception( fmt::format("Illegal type {} of argument of function {}", arguments[0]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[1])->isInteger()) + if (!arguments[1]->isInteger()) throw Exception( fmt::format("Illegal type {} of second argument of function {}", arguments[1]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!removeNullable(arguments[2])->isString()) + if (!arguments[2]->isString()) throw Exception( fmt::format("Illegal type {} of third argument of function {}", arguments[2]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -4037,7 +4037,7 @@ class PadUTF8Impl : public IFunction void tidbExecutePadUTF8(Block & block, const ColumnNumbers & arguments, const size_t result) const { - TypeIndex type_index = removeNullable(block.getByPosition(arguments[1]).type)->getTypeId(); + TypeIndex type_index = block.getByPosition(arguments[1]).type->getTypeId(); switch (type_index) { case TypeIndex::UInt8: @@ -4814,22 +4814,19 @@ class FunctionFormatWithLocale : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - auto first_argument = removeNullable(arguments[0]); - if (!first_argument->isNumber() && !first_argument->isDecimal()) + if (!arguments[0]->isNumber() && !arguments[0]->isDecimal()) throw Exception( - fmt::format("Illegal type {} of first argument of function {}", first_argument->getName(), getName()), + fmt::format("Illegal type {} of first argument of function {}", arguments[0]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - auto second_argument = removeNullable(arguments[1]); - if (!second_argument->isInteger()) + if (!arguments[1]->isInteger()) throw Exception( - fmt::format("Illegal type {} of second argument of function {}", second_argument->getName(), getName()), + fmt::format("Illegal type {} of second argument of function {}", arguments[1]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - auto third_argument = removeNullable(arguments[2]); - if (!third_argument->isString()) + if (!arguments[2]->isString()) throw Exception( - fmt::format("Illegal type {} of third argument of function {}", third_argument->getName(), getName()), + fmt::format("Illegal type {} of third argument of function {}", arguments[2]->getName(), getName()), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto return_type = std::make_shared(); diff --git a/dbms/src/Functions/tests/gtest_datetime_last_day.cpp b/dbms/src/Functions/tests/gtest_datetime_last_day.cpp new file mode 100644 index 00000000000..ee347dd2af1 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_datetime_last_day.cpp @@ -0,0 +1,103 @@ +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +namespace tests +{ +class TestLastDay : public DB::tests::FunctionTest +{ +}; + +TEST_F(TestLastDay, BasicTest) +try +{ + const String func_name = TiDBLastDayTransformerImpl::name; + + // Ignore invalid month error + DAGContext * dag_context = context.getDAGContext(); + UInt64 ori_flags = dag_context->getFlags(); + dag_context->addFlag(TiDBSQLFlags::TRUNCATE_AS_WARNING); + dag_context->clearWarnings(); + + // nullable column test + ASSERT_COLUMN_EQ( + createColumn>({MyDate{2001, 2, 28}.toPackedUInt(), + MyDate{2000, 2, 29}.toPackedUInt(), + MyDate{2000, 6, 30}.toPackedUInt(), + MyDate{2000, 5, 31}.toPackedUInt(), + {}}), + executeFunction(func_name, + {createColumn({MyDate{2001, 2, 10}.toPackedUInt(), + MyDate{2000, 2, 10}.toPackedUInt(), + MyDate{2000, 6, 10}.toPackedUInt(), + MyDate{2000, 5, 10}.toPackedUInt(), + MyDate{2000, 0, 10}.toPackedUInt()})})); + + ASSERT_COLUMN_EQ( + createColumn>({MyDate{2001, 2, 28}.toPackedUInt(), + MyDate{2000, 2, 29}.toPackedUInt(), + MyDate{2000, 6, 30}.toPackedUInt(), + MyDate{2000, 5, 31}.toPackedUInt(), + {}}), + executeFunction(func_name, + {createColumn({MyDateTime{2001, 2, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 2, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 6, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 5, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 0, 10, 10, 10, 10, 0}.toPackedUInt()})})); + + // const test + UInt64 input[] = { + MyDateTime{2001, 2, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 2, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 6, 10, 10, 10, 10, 0}.toPackedUInt(), + MyDateTime{2000, 5, 10, 10, 10, 10, 0}.toPackedUInt(), + }; + + UInt64 output[] = { + MyDate{2001, 2, 28}.toPackedUInt(), + MyDate{2000, 2, 29}.toPackedUInt(), + MyDate{2000, 6, 30}.toPackedUInt(), + MyDate{2000, 5, 31}.toPackedUInt(), + }; + + for (size_t i = 0; i < sizeof(input) / sizeof(UInt64); ++i) + { + ASSERT_COLUMN_EQ( + createConstColumn>(3, output[i]), + executeFunction(func_name, + {createConstColumn(3, input[i])})); + + ASSERT_COLUMN_EQ( + createConstColumn>(3, output[i]), + executeFunction(func_name, + {createConstColumn>(3, input[i])})); + } + + // const nullable test + ASSERT_COLUMN_EQ( + createConstColumn>(3, {}), + executeFunction(func_name, + {createConstColumn>(3, {})})); + + // special const test, month is zero. + ASSERT_COLUMN_EQ( + createConstColumn>(3, {}), + executeFunction(func_name, + {createConstColumn(3, MyDateTime{2000, 0, 10, 10, 10, 10, 0}.toPackedUInt())})); + + dag_context->setFlags(ori_flags); +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index ad261b18fa4..837ae4dc06c 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -208,21 +208,21 @@ void InterpreterSelectQuery::getAndLockStorageWithSchemaVersion(const String & d auto get_and_lock_storage = [&](bool schema_synced) -> std::tuple { /// Get storage in case it's dropped then re-created. // If schema synced, call getTable without try, leading to exception on table not existing. - auto storage_ = schema_synced ? context.getTable(database_name, table_name) : context.tryGetTable(database_name, table_name); - if (!storage_) + auto storage_tmp = schema_synced ? context.getTable(database_name, table_name) : context.tryGetTable(database_name, table_name); + if (!storage_tmp) return std::make_tuple(nullptr, nullptr, DEFAULT_UNSPECIFIED_SCHEMA_VERSION, false); - const auto managed_storage = std::dynamic_pointer_cast(storage_); + const auto managed_storage = std::dynamic_pointer_cast(storage_tmp); if (!managed_storage || !(managed_storage->engineType() == ::TiDB::StorageEngine::TMT || managed_storage->engineType() == ::TiDB::StorageEngine::DT)) { - throw Exception("Specifying schema_version for storage: " + storage_->getName() + throw Exception("Specifying schema_version for storage: " + storage_tmp->getName() + ", table: " + qualified_name + " is not allowed", ErrorCodes::LOGICAL_ERROR); } /// Lock storage. - auto lock = storage_->lockForShare(context.getCurrentQueryId()); + auto lock = storage_tmp->lockForShare(context.getCurrentQueryId()); /// Check schema version, requiring TiDB/TiSpark and TiFlash both use exactly the same schema. // We have three schema versions, two in TiFlash: @@ -238,20 +238,20 @@ void InterpreterSelectQuery::getAndLockStorageWithSchemaVersion(const String & d // From now on we have storage <= query. // If schema was synced, it implies that global >= query, as mentioned above we have storage <= query, we are OK to serve. if (schema_synced) - return std::make_tuple(storage_, lock, storage_schema_version, true); + return std::make_tuple(storage_tmp, lock, storage_schema_version, true); // From now on the schema was not synced. // 1. storage == query, TiDB/TiSpark is using exactly the same schema that altered this table, we are just OK to serve. // 2. global >= query, TiDB/TiSpark is using a schema older than TiFlash global, but as mentioned above we have storage <= query, // meaning that the query schema is still newer than the time when this table was last altered, so we still OK to serve. if (storage_schema_version == query_schema_version || global_schema_version >= query_schema_version) - return std::make_tuple(storage_, lock, storage_schema_version, true); + return std::make_tuple(storage_tmp, lock, storage_schema_version, true); // From now on we have global < query. // Return false for outer to sync and retry. return std::make_tuple(nullptr, nullptr, storage_schema_version, false); }; /// Try get storage and lock once. - StoragePtr storage_; + StoragePtr storage_tmp; TableLockHolder lock; Int64 storage_schema_version; auto log_schema_version = [&](const String & result) { @@ -260,11 +260,11 @@ void InterpreterSelectQuery::getAndLockStorageWithSchemaVersion(const String & d }; bool ok; { - std::tie(storage_, lock, storage_schema_version, ok) = get_and_lock_storage(false); + std::tie(storage_tmp, lock, storage_schema_version, ok) = get_and_lock_storage(false); if (ok) { log_schema_version("OK, no syncing required."); - storage = storage_; + storage = storage_tmp; table_lock = lock; return; } @@ -278,11 +278,11 @@ void InterpreterSelectQuery::getAndLockStorageWithSchemaVersion(const String & d auto schema_sync_cost = std::chrono::duration_cast(Clock::now() - start_time).count(); LOG_DEBUG(log, __PRETTY_FUNCTION__ << " Table " << qualified_name << " schema sync cost " << schema_sync_cost << "ms."); - std::tie(storage_, lock, storage_schema_version, ok) = get_and_lock_storage(true); + std::tie(storage_tmp, lock, storage_schema_version, ok) = get_and_lock_storage(true); if (ok) { log_schema_version("OK after syncing."); - storage = storage_; + storage = storage_tmp; table_lock = lock; return; } @@ -683,7 +683,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). - subquery_settings.extremes = 0; + subquery_settings.extremes = false; subquery_context.setSettings(subquery_settings); interpreter_subquery = std::make_unique( @@ -786,6 +786,15 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline if (!request_str.empty()) { + TableID table_id = InvalidTableID; + if (auto managed_storage = std::dynamic_pointer_cast(storage); managed_storage) + { + table_id = managed_storage->getTableInfo().id; + } + else + { + throw Exception("Not supported request on non-manageable storage"); + } Poco::JSON::Parser parser; Poco::Dynamic::Var result = parser.parse(request_str); auto obj = result.extract(); @@ -798,11 +807,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline ::metapb::Region region; ::google::protobuf::TextFormat::ParseFromString(str, ®ion); - RegionQueryInfo info; - info.region_id = region.id(); const auto & epoch = region.region_epoch(); - info.version = epoch.version(); - info.conf_version = epoch.conf_ver(); + RegionQueryInfo info(region.id(), epoch.version(), epoch.conf_ver(), table_id); if (const auto & managed_storage = std::dynamic_pointer_cast(storage)) { // Extract the handle range according to current table @@ -1066,7 +1072,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha } -void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression) // NOLINT { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression, nullptr); @@ -1167,7 +1173,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) } -void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression) // NOLINT { pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression, nullptr); @@ -1241,7 +1247,7 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) } -void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) +void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) // NOLINT { if (!query.limit_by_value || !query.limit_by_expression_list) return; @@ -1270,7 +1276,7 @@ bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query) auto query_table = query.table(); if (query_table) { - auto ast_union = typeid_cast(query_table.get()); + const auto * ast_union = typeid_cast(query_table.get()); if (ast_union) { for (const auto & elem : ast_union->list_of_selects->children) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 2fb831359fe..3e11a9eeea3 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -258,8 +258,8 @@ struct Settings M(SettingUInt64, dt_segment_delta_small_column_file_size, 8388608, "Determine whether a column file in delta is small or not. 8MB by default.") \ M(SettingUInt64, dt_segment_stable_pack_rows, DEFAULT_MERGE_BLOCK_SIZE, "Expected stable pack rows in DeltaTree Engine.") \ M(SettingFloat, dt_segment_wait_duration_factor, 1, "The factor of wait duration in a write stall.") \ - M(SettingUInt64, dt_bg_gc_check_interval, 600, "Background gc thread check interval, the unit is second.") \ - M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 15, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ + M(SettingUInt64, dt_bg_gc_check_interval, 5, "Background gc thread check interval, the unit is second.") \ + M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 100, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ M(SettingFloat, dt_bg_gc_ratio_threhold_to_trigger_gc, 1.2, "Trigger segment's gc when the ratio of invalid version exceed this threhold. Values smaller than or equal to 1.0 means gc all " \ "segments") \ M(SettingFloat, dt_bg_gc_delta_delete_ratio_to_trigger_gc, 0.3, "Trigger segment's gc when the ratio of delta delete range to stable exceeds this ratio.") \ diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp index f1a8b64d3af..fc8534b068f 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp @@ -60,7 +60,7 @@ DMFileWriter::DMFileWriter(const DMFilePtr & dmfile_, // TODO: currently we only generate index for Integers, Date, DateTime types, and this should be configurable by user. // TODO: If column type is nullable, we won't generate index for it /// for handle column always generate index - bool do_index = cd.id == EXTRA_HANDLE_COLUMN_ID || cd.type->isInteger() || cd.type->isDateOrDateTime(); + bool do_index = cd.id == EXTRA_HANDLE_COLUMN_ID || ((cd.type->isInteger() || cd.type->isDateOrDateTime()) && !cd.type->isNullable()); if (options.flags.isSingleFile()) { diff --git a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp index 770d7e005fb..3d949996f9a 100644 --- a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp +++ b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp @@ -22,8 +22,10 @@ String astToDebugString(const IAST * const ast) // Useless for production env void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & define) { - std::function castDefaultValue; // for lazy bind - castDefaultValue = [&](const Field & value, const DataTypePtr & type) -> Field { + std::function cast_default_value; // for lazy bind + cast_default_value = [&](const Field & value, const DataTypePtr & type) -> Field { + if (value.isNull()) + return value; switch (type->getTypeId()) { case TypeIndex::Float32: @@ -79,7 +81,7 @@ void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & de time_t time = 0; ReadBufferFromMemory buf(date.data(), date.size()); readDateTimeText(time, buf); - return toField((Int64)time); + return toField(static_cast(time)); } case TypeIndex::Decimal32: { @@ -118,14 +120,6 @@ void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & de UInt64 res = applyVisitor(FieldVisitorConvertToNumber(), value); return toField(res); } - case TypeIndex::Nullable: - { - if (value.isNull()) - return value; - auto nullable = std::dynamic_pointer_cast(type); - DataTypePtr nested_type = nullable->getNestedType(); - return castDefaultValue(value, nested_type); // Recursive call on nested type - } default: throw Exception("Unsupported to setColumnDefineDefaultValue with data type: " + type->getName() + " value: " + applyVisitor(FieldVisitorToString(), value) + ", type: " + value.getTypeName()); @@ -139,12 +133,12 @@ void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & de // a cast function // change column_define.default_value - if (auto default_literal = typeid_cast(command.default_expression.get()); + if (const auto * default_literal = typeid_cast(command.default_expression.get()); default_literal && default_literal->value.getType() == Field::Types::String) { define.default_value = default_literal->value; } - else if (auto default_cast_expr = typeid_cast(command.default_expression.get()); + else if (const auto * default_cast_expr = typeid_cast(command.default_expression.get()); default_cast_expr && default_cast_expr->name == "CAST" /* ParserCastExpression::name */) { // eg. CAST('1.234' AS Float32); CAST(999 AS Int32) @@ -153,10 +147,10 @@ void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & de throw Exception("Unknown CAST expression in default expr", ErrorCodes::NOT_IMPLEMENTED); } - auto default_literal_in_cast = typeid_cast(default_cast_expr->arguments->children[0].get()); + const auto * default_literal_in_cast = typeid_cast(default_cast_expr->arguments->children[0].get()); if (default_literal_in_cast) { - Field default_value = castDefaultValue(default_literal_in_cast->value, define.type); + Field default_value = cast_default_value(default_literal_in_cast->value, define.type); define.default_value = default_value; } else @@ -176,16 +170,13 @@ void setColumnDefineDefaultValue(const AlterCommand & command, ColumnDefine & de } catch (const Poco::Exception & e) { - DB::Exception ex(e); - ex.addMessage(fmt::format("(in setColumnDefineDefaultValue for default_expression: {})", astToDebugString(command.default_expression.get()))); - throw ex; + throw DB::Exception(e, fmt::format("(in setColumnDefineDefaultValue for default_expression: {})", astToDebugString(command.default_expression.get()))); } catch (std::exception & e) { - DB::Exception ex( + throw DB::Exception( fmt::format("std::exception: {} (in setColumnDefineDefaultValue for default_expression: {})", e.what(), astToDebugString(command.default_expression.get())), ErrorCodes::LOGICAL_ERROR); - throw ex; } } } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp index f44dd5d3163..ff036b5a75e 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp @@ -302,7 +302,7 @@ CATCH TEST(StorageDeltaMergeInternalTest, GetMergedQueryRanges) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = GET_REGION_RANGE(100, 200, 1); regions.emplace_back(region); region.range_in_table = GET_REGION_RANGE(200, 250, 1); @@ -322,7 +322,7 @@ TEST(StorageDeltaMergeInternalTest, GetMergedQueryRanges) TEST(StorageDeltaMergeInternalTest, GetMergedQueryRangesCommonHandle) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(100, 200, 2).toRegionRange(1); regions.emplace_back(region); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(200, 250, 2).toRegionRange(1); @@ -342,7 +342,7 @@ TEST(StorageDeltaMergeInternalTest, GetMergedQueryRangesCommonHandle) TEST(StorageDeltaMergeInternalTest, MergedUnsortedQueryRanges) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = GET_REGION_RANGE(2360148, 2456148, 1); regions.emplace_back(region); region.range_in_table = GET_REGION_RANGE(1961680, 2057680, 1); @@ -372,7 +372,7 @@ TEST(StorageDeltaMergeInternalTest, MergedUnsortedQueryRanges) TEST(StorageDeltaMergeInternalTest, MergedUnsortedQueryRangesCommonHandle) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(2360148, 2456148, 2).toRegionRange(1); regions.emplace_back(region); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(1961680, 2057680, 2).toRegionRange(1); @@ -402,7 +402,7 @@ TEST(StorageDeltaMergeInternalTest, MergedUnsortedQueryRangesCommonHandle) TEST(StorageDeltaMergeInternalTest, GetFullQueryRanges) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = GET_REGION_RANGE(std::numeric_limits::min(), std::numeric_limits::max(), 1); regions.emplace_back(region); @@ -415,7 +415,7 @@ TEST(StorageDeltaMergeInternalTest, GetFullQueryRanges) TEST(StorageDeltaMergeInternalTest, OverlapQueryRanges) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = GET_REGION_RANGE(100, 200, 1); regions.emplace_back(region); region.range_in_table = GET_REGION_RANGE(150, 250, 1); @@ -432,7 +432,7 @@ TEST(StorageDeltaMergeInternalTest, OverlapQueryRanges) TEST(StorageDeltaMergeInternalTest, OverlapQueryRangesCommonHandle) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(100, 200, 2).toRegionRange(1); regions.emplace_back(region); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(150, 250, 2).toRegionRange(1); @@ -450,7 +450,7 @@ TEST(StorageDeltaMergeInternalTest, WeirdRange) { // [100, 200), [200, MAX] MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = GET_REGION_RANGE(100, 200, 1); regions.emplace_back(region); region.range_in_table = GET_REGION_RANGE(200, std::numeric_limits::max(), 1); @@ -465,7 +465,7 @@ TEST(StorageDeltaMergeInternalTest, WeirdRangeCommonHandle) { // [100, 200), [200, MAX), [MAX, MAX) MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(100, 200, 2).toRegionRange(1); regions.emplace_back(region); region.range_in_table @@ -484,7 +484,7 @@ TEST(StorageDeltaMergeInternalTest, RangeSplit) { { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(100, 200, 2).toRegionRange(1); regions.emplace_back(region); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(200, 300, 2).toRegionRange(1); @@ -532,7 +532,7 @@ TEST(StorageDeltaMergeInternalTest, RangeSplit) { MvccQueryInfo::RegionsQueryInfo regions; - RegionQueryInfo region; + RegionQueryInfo region(1, 1, 1, 1); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(0, 100, 2).toRegionRange(1); regions.emplace_back(region); region.range_in_table = DMTestEnv::getRowKeyRangeForClusteredIndex(200, 300, 2).toRegionRange(1); @@ -671,7 +671,10 @@ try } else if (iter.name == EXTRA_TABLE_ID_COLUMN_NAME) { - ASSERT_EQ(c->getInt(i), 1); + Field res; + c->get(i, res); + ASSERT(!res.isNull()); + ASSERT(res.get() == 1); } } } diff --git a/dbms/src/Storages/MutableSupport.cpp b/dbms/src/Storages/MutableSupport.cpp index 0f10d8014a3..10f67f16272 100644 --- a/dbms/src/Storages/MutableSupport.cpp +++ b/dbms/src/Storages/MutableSupport.cpp @@ -16,7 +16,8 @@ const DataTypePtr MutableSupport::tidb_pk_column_int_type = DataTypeFactory::ins const DataTypePtr MutableSupport::tidb_pk_column_string_type = DataTypeFactory::instance().get("String"); const DataTypePtr MutableSupport::version_column_type = DataTypeFactory::instance().get("UInt64"); const DataTypePtr MutableSupport::delmark_column_type = DataTypeFactory::instance().get("UInt8"); -const DataTypePtr MutableSupport::extra_table_id_column_type = DataTypeFactory::instance().get("Int64"); +/// it should not be nullable, but TiDB does not set not null flag for extra_table_id_column_type, so has to align with TiDB +const DataTypePtr MutableSupport::extra_table_id_column_type = DataTypeFactory::instance().get("Nullable(Int64)"); ; } // namespace DB diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index d018094860f..5145cc03a33 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -752,14 +752,6 @@ void BlobStore::BlobStats::restore() } // restore `roll_id` roll_id = max_restored_file_id + 1; - // restore `old_ids` - for (BlobFileId old_id = 1; old_id < roll_id; ++old_id) - { - if (existing_file_ids.count(old_id) == 0) - { - old_ids.emplace_back(old_id); - } - } } std::lock_guard BlobStore::BlobStats::lock() const @@ -813,7 +805,6 @@ BlobStatPtr BlobStore::BlobStats::createStatNotCheckingRoll(BlobFileId blob_file void BlobStore::BlobStats::eraseStat(const BlobStatPtr && stat, const std::lock_guard &) { - old_ids.emplace_back(stat->id); stats_map.remove(stat); } @@ -841,24 +832,6 @@ void BlobStore::BlobStats::eraseStat(BlobFileId blob_file_id, const std::lock_gu eraseStat(std::move(stat), lock); } -BlobFileId BlobStore::BlobStats::chooseNewStat() -{ - /** - * If we do have any `old blob id` which may removed by GC. - * Then we should get a `old blob id` rather than create a new blob id. - * If `old_ids` is empty , we will use the `roll_id` as the new - * id return. After roll_id generate a `BlobStat`, it will been `++`. - */ - if (old_ids.empty()) - { - return roll_id; - } - - auto rv = old_ids.front(); - old_ids.pop_front(); - return rv; -} - std::pair BlobStore::BlobStats::chooseStat(size_t buf_size, UInt64 file_limit_size, const std::lock_guard &) { BlobStatPtr stat_ptr = nullptr; @@ -867,7 +840,7 @@ std::pair BlobStore::BlobStats::chooseStat(size_t buf_s // No stats exist if (stats_map.empty()) { - return std::make_pair(nullptr, chooseNewStat()); + return std::make_pair(nullptr, roll_id); } for (const auto & stat : stats_map) @@ -884,7 +857,7 @@ std::pair BlobStore::BlobStats::chooseStat(size_t buf_s if (!stat_ptr) { - return std::make_pair(nullptr, chooseNewStat()); + return std::make_pair(nullptr, roll_id); } return std::make_pair(stat_ptr, INVALID_BLOBFILE_ID); diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index 9867290c536..39fd3344f4f 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -143,8 +143,6 @@ class BlobStore : private Allocator */ std::pair chooseStat(size_t buf_size, UInt64 file_limit_size, const std::lock_guard &); - BlobFileId chooseNewStat(); - BlobStatPtr blobIdToStat(BlobFileId file_id, bool restore_if_not_exist = false); std::list getStats() const @@ -167,7 +165,6 @@ class BlobStore : private Allocator BlobStore::Config config; BlobFileId roll_id = 1; - std::list old_ids; std::list stats_map; mutable std::mutex lock_stats; }; diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index 94df459b025..12fe2cef2ed 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -63,8 +63,12 @@ void VersionedPageEntries::createNewEntry(const PageVersionType & ver, const Pag if (type == EditRecordType::VAR_ENTRY) { - auto last_iter = entries.rbegin(); - if (last_iter->second.isDelete()) + auto last_iter = MapUtils::findLess(entries, PageVersionType(ver.sequence + 1, 0)); + if (last_iter == entries.end()) + { + entries.emplace(ver, EntryOrDelete::newNormalEntry(entry)); + } + else if (last_iter->second.isDelete()) { entries.emplace(ver, EntryOrDelete::newNormalEntry(entry)); } @@ -302,10 +306,8 @@ VersionedPageEntries::resolveToPageId(UInt64 seq, bool check_prev, PageEntryV3 * } else if (type == EditRecordType::VAR_EXTERNAL) { - // If we applied write batches like this: [ver=1]{putExternal 10}, [ver=2]{ref 11->10, del 10} - // then by ver=2, we should not able to read 10, but able to read 11 (resolving 11 ref to 10). - // when resolving 11 to 10, we need to set `check_prev` to true - bool ok = !is_deleted || (is_deleted && (check_prev ? (seq <= delete_ver.sequence) : (seq < delete_ver.sequence))); + // We may add reference to an external id even if it is logically deleted. + bool ok = check_prev ? true : (!is_deleted || (is_deleted && seq < delete_ver.sequence)); if (create_ver.sequence <= seq && ok) { return {RESOLVE_TO_NORMAL, buildV3Id(0, 0), PageVersionType(0)}; @@ -354,8 +356,9 @@ Int64 VersionedPageEntries::incrRefCount(const PageVersionType & ver) } else if (type == EditRecordType::VAR_EXTERNAL) { - if (create_ver <= ver && (!is_deleted || (is_deleted && ver < delete_ver))) + if (create_ver <= ver) { + // We may add reference to an external id even if it is logically deleted. return ++being_ref_count; } } @@ -397,7 +400,6 @@ PageSize VersionedPageEntries::getEntriesByBlobIds( bool VersionedPageEntries::cleanOutdatedEntries( UInt64 lowest_seq, - PageIdV3Internal page_id, std::map> * normal_entries_to_deref, PageEntriesV3 & entries_removed, const PageLock & /*page_lock*/) @@ -416,15 +418,6 @@ bool VersionedPageEntries::cleanOutdatedEntries( // need to decrease the ref count by second.origin_page_id, ver=iter->first, num=1> if (auto [deref_counter, new_created] = normal_entries_to_deref->emplace(std::make_pair(ori_page_id, std::make_pair(/*ver=*/create_ver, /*count=*/1))); !new_created) { - if (deref_counter->second.first.sequence != create_ver.sequence) - { - throw Exception(fmt::format( - "There exist two different version of ref, should not happen [page_id={}] [ori_page_id={}] [ver={}] [another_ver={}]", - page_id, - ori_page_id, - create_ver, - deref_counter->second.first)); - } // the id is already exist in deref map, increase the num to decrease ref count deref_counter->second.second += 1; } @@ -508,7 +501,8 @@ bool VersionedPageEntries::derefAndClean(UInt64 lowest_seq, PageIdV3Internal pag } else if (type == EditRecordType::VAR_ENTRY) { - // decrease the ref-counter + // Decrease the ref-counter. The entry may be moved to a newer entry with same sequence but higher epoch, + // so we need to find the one less than and decrease the ref-counter of it. auto iter = MapUtils::findMutLess(entries, PageVersionType(deref_ver.sequence + 1, 0)); if (iter == entries.end()) { @@ -530,7 +524,7 @@ bool VersionedPageEntries::derefAndClean(UInt64 lowest_seq, PageIdV3Internal pag // Clean outdated entries after decreased the ref-counter // set `normal_entries_to_deref` to be nullptr to ignore cleaning ref-var-entries - return cleanOutdatedEntries(lowest_seq, page_id, /*normal_entries_to_deref*/ nullptr, entries_removed, page_lock); + return cleanOutdatedEntries(lowest_seq, /*normal_entries_to_deref*/ nullptr, entries_removed, page_lock); } throw Exception(fmt::format("calling derefAndClean with invalid state [state={}]", toDebugString())); @@ -863,7 +857,10 @@ void PageDirectory::applyRefEditRecord( const VersionedPageEntriesPtr & resolve_version_list = resolve_ver_iter->second; // If we already hold the lock from `id_to_resolve`, then we should not request it again. // This can happen when `id_to_resolve` have other operating in current writebatch - auto [need_collapse, next_id_to_resolve, next_ver_to_resolve] = resolve_version_list->resolveToPageId(ver_to_resolve.sequence, false, nullptr); + auto [need_collapse, next_id_to_resolve, next_ver_to_resolve] = resolve_version_list->resolveToPageId( + ver_to_resolve.sequence, + /*check_prev=*/true, + nullptr); switch (need_collapse) { case VersionedPageEntries::RESOLVE_FAIL: @@ -966,7 +963,7 @@ void PageDirectory::apply(PageEntriesEdit && edit, const WriteLimiterPtr & write case EditRecordType::VAR_ENTRY: case EditRecordType::VAR_EXTERNAL: case EditRecordType::VAR_REF: - throw Exception(fmt::format("should not handle {} edit", r.type)); + throw Exception(fmt::format("should not handle edit with invalid type [type={}]", r.type)); } } catch (DB::Exception & e) @@ -1000,7 +997,7 @@ void PageDirectory::gcApply(PageEntriesEdit && migrated_edit, const WriteLimiter iter = mvcc_table_directory.find(record.page_id); if (unlikely(iter == mvcc_table_directory.end())) { - throw Exception(fmt::format("Can't found [page_id={}] while doing gcApply", record.page_id), ErrorCodes::LOGICAL_ERROR); + throw Exception(fmt::format("Can't find [page_id={}] while doing gcApply", record.page_id), ErrorCodes::LOGICAL_ERROR); } } // release the read lock on `table_rw_mutex` @@ -1126,7 +1123,6 @@ PageEntriesV3 PageDirectory::gcInMemEntries() // do gc on the version list without lock on `mvcc_table_directory`. const bool all_deleted = iter->second->cleanOutdatedEntries( lowest_seq, - /*page_id=*/iter->first, &normal_entries_to_deref, all_del_entries, iter->second->acquireLock()); @@ -1161,8 +1157,8 @@ PageEntriesV3 PageDirectory::gcInMemEntries() const bool all_deleted = iter->second->derefAndClean( lowest_seq, page_id, - deref_counter.first, - deref_counter.second, + /*deref_ver=*/deref_counter.first, + /*deref_count=*/deref_counter.second, all_del_entries); if (all_deleted) diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index a148a03bab8..c8f9af8e8ea 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -17,8 +18,6 @@ #include #include -#include "Encryption/FileProvider.h" - namespace CurrentMetrics { extern const Metric PSMVCCNumSnapshots; @@ -205,7 +204,6 @@ class VersionedPageEntries */ bool cleanOutdatedEntries( UInt64 lowest_seq, - PageIdV3Internal page_id, std::map> * normal_entries_to_deref, PageEntriesV3 & entries_removed, const PageLock & page_lock); diff --git a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp index e5427d6e562..d040e9fd378 100644 --- a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp @@ -12,17 +12,35 @@ PageDirectoryPtr PageDirectoryFactory::create(FileProviderPtr & file_provider, P auto [wal, reader] = WALStore::create(file_provider, delegator); PageDirectoryPtr dir = std::make_unique(std::move(wal)); loadFromDisk(dir, std::move(reader)); - // TODO: After restored ends, set the last offset of log file for `wal` - if (blob_stats) - blob_stats->restore(); // Reset the `sequence` to the maximum of persisted. dir->sequence = max_applied_ver.sequence; + + if (blob_stats) + { + // After all entries restored to `mvcc_table_directory`, only apply + // the latest entry to `blob_stats`, or we may meet error since + // some entries may be removed in memory but not get compacted + // in the log file. + for (const auto & [page_id, entries] : dir->mvcc_table_directory) + { + (void)page_id; + if (auto entry = entries->getEntry(max_applied_ver.sequence); entry) + { + blob_stats->restoreByEntry(*entry); + } + } + + blob_stats->restore(); + } + + // TODO: After restored ends, set the last offset of log file for `wal` return dir; } PageDirectoryPtr PageDirectoryFactory::createFromEdit(FileProviderPtr & file_provider, PSDiskDelegatorPtr & delegator, const PageEntriesEdit & edit) { auto [wal, reader] = WALStore::create(file_provider, delegator); + (void)reader; PageDirectoryPtr dir = std::make_unique(std::move(wal)); loadEdit(dir, edit); if (blob_stats) @@ -67,8 +85,6 @@ void PageDirectoryFactory::loadEdit(const PageDirectoryPtr & dir, const PageEntr } case EditRecordType::VAR_ENTRY: version_list->fromRestored(r); - if (blob_stats) - blob_stats->restoreByEntry(r.entry); break; case EditRecordType::PUT_EXTERNAL: { @@ -82,8 +98,6 @@ void PageDirectoryFactory::loadEdit(const PageDirectoryPtr & dir, const PageEntr } case EditRecordType::PUT: version_list->createNewEntry(restored_version, r.entry); - if (blob_stats) - blob_stats->restoreByEntry(r.entry); break; case EditRecordType::DEL: case EditRecordType::VAR_DELETE: // nothing different from `DEL` @@ -94,8 +108,6 @@ void PageDirectoryFactory::loadEdit(const PageDirectoryPtr & dir, const PageEntr break; case EditRecordType::UPSERT: version_list->createNewEntry(restored_version, r.entry); - if (blob_stats) - blob_stats->restoreByEntry(r.entry); break; } } diff --git a/dbms/src/Storages/Page/V3/PageEntriesEdit.h b/dbms/src/Storages/Page/V3/PageEntriesEdit.h index b75910edc46..f5fe35f60c7 100644 --- a/dbms/src/Storages/Page/V3/PageEntriesEdit.h +++ b/dbms/src/Storages/Page/V3/PageEntriesEdit.h @@ -68,6 +68,31 @@ enum class EditRecordType VAR_DELETE, }; +inline const char * typeToString(EditRecordType t) +{ + switch (t) + { + case EditRecordType::PUT: + return "PUT "; + case EditRecordType::PUT_EXTERNAL: + return "EXT "; + case EditRecordType::REF: + return "REF "; + case EditRecordType::DEL: + return "DEL "; + case EditRecordType::UPSERT: + return "UPSERT "; + case EditRecordType::VAR_ENTRY: + return "VAR_ENT"; + case EditRecordType::VAR_REF: + return "VAR_REF"; + case EditRecordType::VAR_EXTERNAL: + return "VAR_EXT"; + case EditRecordType::VAR_DELETE: + return "VAR_DEL"; + } +} + /// Page entries change to apply to PageDirectory class PageEntriesEdit { @@ -176,10 +201,28 @@ class PageEntriesEdit PageIdV3Internal ori_page_id; PageVersionType version; PageEntryV3 entry; - Int64 being_ref_count = 1; + Int64 being_ref_count; + + EditRecord() + : page_id(0) + , ori_page_id(0) + , being_ref_count(1) + {} }; using EditRecords = std::vector; + static String toDebugString(const EditRecord & rec) + { + return fmt::format( + "{{type:{}, page_id:{}, ori_id:{}, version:{}, entry:{}, being_ref_count:{}}}", + typeToString(rec.type), + rec.page_id, + rec.ori_page_id, + rec.version, + DB::PS::V3::toDebugString(rec.entry), + rec.being_ref_count); + } + void appendRecord(const EditRecord & rec) { records.emplace_back(rec); diff --git a/dbms/src/Storages/Page/V3/WAL/WALReader.cpp b/dbms/src/Storages/Page/V3/WAL/WALReader.cpp index d4d8b31de84..63974f76eba 100644 --- a/dbms/src/Storages/Page/V3/WAL/WALReader.cpp +++ b/dbms/src/Storages/Page/V3/WAL/WALReader.cpp @@ -79,9 +79,18 @@ WALStoreReader::findCheckpoint(LogFilenameSet && all_files) LogFilename latest_checkpoint = *latest_checkpoint_iter; for (auto iter = all_files.cbegin(); iter != all_files.cend(); /*empty*/) { - if (iter->log_num < latest_checkpoint.log_num) + // We use as the checkpoint, so all files less than or equal + // to latest_checkpoint.log_num can be erase + if (iter->log_num <= latest_checkpoint.log_num) { - // TODO: clean useless file that is older than `checkpoint` + if (iter->log_num == latest_checkpoint.log_num && iter->level_num != 0) + { + // the checkpoint file, not remove + } + else + { + // TODO: clean useless file that is older than `checkpoint` + } iter = all_files.erase(iter); } else @@ -186,6 +195,7 @@ bool WALStoreReader::openNextFile() if (!checkpoint_read_done) { do_open(*checkpoint_file); + checkpoint_read_done = true; } else { diff --git a/dbms/src/Storages/Page/V3/WAL/serialize.cpp b/dbms/src/Storages/Page/V3/WAL/serialize.cpp index d51d67bff34..26f3f97e22e 100644 --- a/dbms/src/Storages/Page/V3/WAL/serialize.cpp +++ b/dbms/src/Storages/Page/V3/WAL/serialize.cpp @@ -64,7 +64,7 @@ void serializePutTo(const PageEntriesEdit::EditRecord & record, WriteBuffer & bu { assert(record.type == EditRecordType::PUT || record.type == EditRecordType::UPSERT || record.type == EditRecordType::VAR_ENTRY); - writeIntBinary(EditRecordType::PUT, buf); + writeIntBinary(record.type, buf); UInt32 flags = 0; writeIntBinary(flags, buf); diff --git a/dbms/src/Storages/Page/V3/WALStore.cpp b/dbms/src/Storages/Page/V3/WALStore.cpp index b9fa1274c06..fea7a1c8361 100644 --- a/dbms/src/Storages/Page/V3/WALStore.cpp +++ b/dbms/src/Storages/Page/V3/WALStore.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -114,10 +115,22 @@ std::tuple, LogFilename> WALStore::createLogWriter( WALStore::FilesSnapshot WALStore::getFilesSnapshot() const { - const auto current_writting_log_num = [this]() { + const auto [ok, current_writting_log_num] = [this]() -> std::tuple { std::lock_guard lock(log_file_mutex); - return log_file->logNumber(); + if (!log_file) + { + return {false, 0}; + } + return {true, log_file->logNumber()}; }(); + // Return empty set if `log_file` is not ready + if (!ok) + { + return WALStore::FilesSnapshot{ + .current_writting_log_num = 0, + .persisted_log_files = {}, + }; + } // Only those files are totally persisted LogFilenameSet persisted_log_files = WALStoreReader::listAllFiles(delegator, logger); @@ -163,12 +176,24 @@ bool WALStore::saveSnapshot(FilesSnapshot && files_snap, PageEntriesEdit && dire LOG_FMT_INFO(logger, "Rename log file to normal done [fullname={}]", normal_fullname); } + // #define ARCHIVE_COMPACTED_LOGS // keep for debug + // Remove compacted log files. for (const auto & filename : files_snap.persisted_log_files) { if (auto f = Poco::File(filename.fullname(LogFileStage::Normal)); f.exists()) { +#ifndef ARCHIVE_COMPACTED_LOGS f.remove(); +#else + const Poco::Path archive_path(delegator->defaultPath(), "archive"); + Poco::File archive_dir(archive_path); + if (!archive_dir.exists()) + archive_dir.createDirectory(); + auto dest = archive_path.toString() + "/" + filename.filename(LogFileStage::Normal); + f.moveTo(dest); + LOG_FMT_INFO(logger, "archive {} to {}", filename.fullname(LogFileStage::Normal), dest); +#endif } } // TODO: Log more information. duration, num entries, size of compact log file... diff --git a/dbms/src/Storages/Page/V3/tests/entries_helper.h b/dbms/src/Storages/Page/V3/tests/entries_helper.h index 00375d16fe3..f2ff6c9e3bc 100644 --- a/dbms/src/Storages/Page/V3/tests/entries_helper.h +++ b/dbms/src/Storages/Page/V3/tests/entries_helper.h @@ -145,7 +145,7 @@ inline ::testing::AssertionResult getEntriesCompare( { // not the expected entry we want String err_msg; - auto expect_expr = fmt::format("Entry at {} [index={}]", idx); + auto expect_expr = fmt::format("Entry at {} [index={}]", idx, idx); auto actual_expr = fmt::format("Get entries {} from {} with snap {} [index={}", page_ids_expr, dir_expr, snap_expr, idx); return testing::internal::EqFailure( expect_expr.c_str(), diff --git a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp index a07e491cc0f..842c05610e7 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp @@ -38,9 +38,7 @@ TEST_F(BlobStoreStatsTest, RestoreEmpty) ASSERT_TRUE(stats_copy.empty()); EXPECT_EQ(stats.roll_id, 1); - auto next_file_id = stats.chooseNewStat(); - EXPECT_EQ(next_file_id, 1); - EXPECT_NO_THROW(stats.createStat(next_file_id, stats.lock())); + EXPECT_NO_THROW(stats.createStat(stats.roll_id, stats.lock())); } TEST_F(BlobStoreStatsTest, Restore) @@ -91,19 +89,9 @@ try // a new file bigger than restored `roll_id` EXPECT_ANY_THROW({ stats.createStat(14, stats.lock()); }); - for (BlobFileId i = 1; i <= 20; ++i) - { - if (i == file_id1 || i == file_id2) - { - EXPECT_ANY_THROW({ stats.createStat(i, stats.lock()); }); - } - else - { - auto new_file_id = stats.chooseNewStat(); - EXPECT_EQ(new_file_id, i); - EXPECT_NO_THROW({ stats.createStat(new_file_id, stats.lock()); }); - } - } + EXPECT_ANY_THROW({ stats.createStat(file_id1, stats.lock()); }); + EXPECT_ANY_THROW({ stats.createStat(file_id2, stats.lock()); }); + EXPECT_ANY_THROW({ stats.createStat(stats.roll_id + 1, stats.lock()); }); } CATCH @@ -125,13 +113,6 @@ TEST_F(BlobStoreStatsTest, testStats) stats.eraseStat(1, stats.lock()); ASSERT_EQ(stats.stats_map.size(), 1); ASSERT_EQ(stats.roll_id, 3); - ASSERT_EQ(stats.old_ids.size(), 2); - - auto old_it = stats.old_ids.begin(); - - ASSERT_EQ((*old_it++), 0); - ASSERT_EQ((*old_it++), 1); - ASSERT_EQ(old_it, stats.old_ids.end()); } @@ -245,11 +226,10 @@ TEST_F(BlobStoreStatsTest, testFullStats) offset = stat->getPosFromStat(BLOBFILE_LIMIT_SIZE - 100); ASSERT_EQ(offset, 100); - // Then choose stat , it should return the stat id 1 - // cause in this time , stat which id is 1 have been earsed, - // and stat which id is 2 is full. + // Then choose stat , it should return the stat id 3 + // Stat which id is 2 is full. std::tie(stat, blob_file_id) = stats.chooseStat(100, BLOBFILE_LIMIT_SIZE, stats.lock()); - ASSERT_EQ(blob_file_id, 1); + ASSERT_EQ(blob_file_id, 3); ASSERT_FALSE(stat); } diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp index d71cb1c78a9..75279c87bbb 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp @@ -508,6 +508,36 @@ TEST_F(PageDirectoryTest, NewRefAfterDel) } } +TEST_F(PageDirectoryTest, RefToExt) +try +{ + { + PageEntriesEdit edit; + edit.putExternal(83); + dir->apply(std::move(edit)); + } + { + PageEntriesEdit edit; + edit.ref(85, 83); + dir->apply(std::move(edit)); + } + { + PageEntriesEdit edit; + edit.del(83); + dir->apply(std::move(edit)); + } + // The external id "83" is not changed, + // we may add ref to external "83" even + // if it is logical delete but have other + // alive reference page. + { + PageEntriesEdit edit; + edit.ref(86, 83); + dir->apply(std::move(edit)); + } +} +CATCH + TEST_F(PageDirectoryTest, NormalPageId) try { @@ -583,16 +613,12 @@ CATCH class VersionedEntriesTest : public ::testing::Test { public: - void SetUp() override - { - } - using DerefCounter = std::map>; std::tuple runClean(UInt64 seq) { DerefCounter deref_counter; PageEntriesV3 removed_entries; - bool all_removed = entries.cleanOutdatedEntries(seq, buildV3Id(TEST_NAMESPACE_ID, page_id), &deref_counter, removed_entries, entries.acquireLock()); + bool all_removed = entries.cleanOutdatedEntries(seq, &deref_counter, removed_entries, entries.acquireLock()); return {all_removed, removed_entries, deref_counter}; } @@ -666,6 +692,15 @@ TEST_F(VersionedEntriesTest, InsertGet) } } +TEST_F(VersionedEntriesTest, InsertWithLowerVersion) +{ + INSERT_ENTRY(5); + ASSERT_SAME_ENTRY(*entries.getEntry(5), entry_v5); + ASSERT_FALSE(entries.getEntry(2).has_value()); + INSERT_ENTRY(2); + ASSERT_SAME_ENTRY(*entries.getEntry(2), entry_v2); +} + TEST_F(VersionedEntriesTest, GC) try { @@ -1554,6 +1589,117 @@ try } CATCH +TEST_F(PageDirectoryGCTest, GCOnRefedEntries2) +try +{ + // 10->entry1, 11->10=>11->entry1; del 10->entry1 + PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + { + PageEntriesEdit edit; + edit.put(10, entry1); + dir->apply(std::move(edit)); + } + { + PageEntriesEdit edit; + edit.ref(11, 10); + dir->apply(std::move(edit)); + } + { + PageEntriesEdit edit; + edit.ref(12, 10); + edit.del(10); + dir->apply(std::move(edit)); + } + // entry1 should not be removed + { + auto outdated_entries = dir->gcInMemEntries(); + EXPECT_TRUE(outdated_entries.empty()); + } + + // del 11->entry1 + { + PageEntriesEdit edit; + edit.del(11); + edit.del(12); + dir->apply(std::move(edit)); + } + // entry1 get removed + { + auto outdated_entries = dir->gcInMemEntries(); + EXPECT_EQ(1, outdated_entries.size()); + EXPECT_SAME_ENTRY(entry1, *outdated_entries.begin()); + } +} +CATCH + +TEST_F(PageDirectoryGCTest, UpsertOnRefedEntries) +try +{ + // 10->entry1, 11->10, 12->10 + PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + { + PageEntriesEdit edit; + edit.put(10, entry1); + dir->apply(std::move(edit)); + } + { + PageEntriesEdit edit; + edit.ref(11, 10); + dir->apply(std::move(edit)); + } + { + PageEntriesEdit edit; + edit.ref(12, 10); + edit.del(10); + dir->apply(std::move(edit)); + } + // entry1 should not be removed + { + auto outdated_entries = dir->gcInMemEntries(); + EXPECT_TRUE(outdated_entries.empty()); + } + + // upsert 10->entry2 + PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + { + PageEntriesEdit edit; + auto full_gc_entries = dir->getEntriesByBlobIds({1}); + auto ids = full_gc_entries.first.at(1); + ASSERT_EQ(ids.size(), 1); + edit.upsertPage(std::get<0>(ids[0]), std::get<1>(ids[0]), entry2); + dir->gcApply(std::move(edit)); + } + + auto removed_entries = dir->gcInMemEntries(); + ASSERT_EQ(removed_entries.size(), 1); + EXPECT_SAME_ENTRY(removed_entries[0], entry1); + + { + auto snap = dir->createSnapshot(); + EXPECT_ENTRY_EQ(entry2, dir, 11, snap); + EXPECT_ENTRY_EQ(entry2, dir, 12, snap); + } + + // del 11->entry2 + { + PageEntriesEdit edit; + edit.del(11); + dir->apply(std::move(edit)); + EXPECT_EQ(dir->gcInMemEntries().size(), 0); + } + // del 12->entry2 + { + PageEntriesEdit edit; + edit.del(12); + dir->apply(std::move(edit)); + // entry2 get removed + auto outdated_entries = dir->gcInMemEntries(); + EXPECT_EQ(1, outdated_entries.size()); + EXPECT_SAME_ENTRY(entry2, *outdated_entries.begin()); + } +} +CATCH + TEST_F(PageDirectoryGCTest, GCOnRefedExternalEntries) try { diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp index cac4a2c89a6..d2b85b29468 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -844,6 +844,47 @@ try } CATCH +TEST_F(PageStorageTest, GcReuseSpaceThenRestore) +try +{ + DB::UInt64 tag = 0; + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_storage->write(std::move(batch)); + } + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_storage->write(std::move(batch)); + } + + { + SCOPED_TRACE("fist gc"); + page_storage->gc(); + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_storage->write(std::move(batch)); + } + + page_storage.reset(); + page_storage = reopenWithConfig(config); +} +CATCH + } // namespace PS::V3::tests } // namespace DB diff --git a/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp index 21d03a7707c..d5ebdab0a11 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp @@ -100,17 +100,17 @@ TEST(WALSeriTest, Upserts) auto deseri_edit = DB::PS::V3::ser::deserializeFrom(DB::PS::V3::ser::serializeTo(edit)); ASSERT_EQ(deseri_edit.size(), 3); auto iter = deseri_edit.getRecords().begin(); - EXPECT_EQ(iter->type, EditRecordType::PUT); // deser as put + EXPECT_EQ(iter->type, EditRecordType::UPSERT); EXPECT_EQ(iter->page_id.low, 1); EXPECT_EQ(iter->version, ver20_1); EXPECT_SAME_ENTRY(iter->entry, entry_p1_2); iter++; - EXPECT_EQ(iter->type, EditRecordType::PUT); // deser as put + EXPECT_EQ(iter->type, EditRecordType::UPSERT); EXPECT_EQ(iter->page_id.low, 3); EXPECT_EQ(iter->version, ver21_1); EXPECT_SAME_ENTRY(iter->entry, entry_p3_2); iter++; - EXPECT_EQ(iter->type, EditRecordType::PUT); // deser as put + EXPECT_EQ(iter->type, EditRecordType::UPSERT); EXPECT_EQ(iter->page_id.low, 5); EXPECT_EQ(iter->version, ver21_1); EXPECT_SAME_ENTRY(iter->entry, entry_p5_2); @@ -212,6 +212,56 @@ class WALStoreTest : public DB::base::TiFlashStorageTestBasic PSDiskDelegatorPtr delegator; }; +TEST_F(WALStoreTest, FindCheckpointFile) +{ + Poco::Logger * log = &Poco::Logger::get("WALStoreTest"); + auto path = getTemporaryPath(); + + { + // no checkpoint + LogFilenameSet files{ + LogFilename::parseFrom(path, "log_1_0", log), + LogFilename::parseFrom(path, "log_2_0", log), + LogFilename::parseFrom(path, "log_3_0", log), + LogFilename::parseFrom(path, "log_4_0", log), + }; + auto [cp, files_to_read] = WALStoreReader::findCheckpoint(std::move(files)); + ASSERT_FALSE(cp.has_value()); + EXPECT_EQ(files_to_read.size(), 4); + } + + { + // checkpoint and some other logfiles + LogFilenameSet files{ + LogFilename::parseFrom(path, "log_12_1", log), + LogFilename::parseFrom(path, "log_13_0", log), + LogFilename::parseFrom(path, "log_14_0", log), + }; + auto [cp, files_to_read] = WALStoreReader::findCheckpoint(std::move(files)); + ASSERT_TRUE(cp.has_value()); + EXPECT_EQ(cp->log_num, 12); + EXPECT_EQ(cp->level_num, 1); + EXPECT_EQ(files_to_read.size(), 2); + } + + { + // some files before checkpoint left on disk + LogFilenameSet files{ + LogFilename::parseFrom(path, "log_10_0", log), + LogFilename::parseFrom(path, "log_11_0", log), + LogFilename::parseFrom(path, "log_12_0", log), + LogFilename::parseFrom(path, "log_12_1", log), + LogFilename::parseFrom(path, "log_13_0", log), + LogFilename::parseFrom(path, "log_14_0", log), + }; + auto [cp, files_to_read] = WALStoreReader::findCheckpoint(std::move(files)); + ASSERT_TRUE(cp.has_value()); + EXPECT_EQ(cp->log_num, 12); + EXPECT_EQ(cp->level_num, 1); + EXPECT_EQ(files_to_read.size(), 2); + } +} + TEST_F(WALStoreTest, Empty) { auto ctx = DB::tests::TiFlashTestEnv::getContext(); @@ -223,6 +273,7 @@ TEST_F(WALStoreTest, Empty) while (reader->remained()) { auto [ok, edit] = reader->next(); + (void)edit; if (!ok) { reader->throwIfError(); diff --git a/dbms/src/Storages/RegionQueryInfo.h b/dbms/src/Storages/RegionQueryInfo.h index cd095e4b83b..0186724c981 100644 --- a/dbms/src/Storages/RegionQueryInfo.h +++ b/dbms/src/Storages/RegionQueryInfo.h @@ -5,14 +5,22 @@ namespace DB { - using DecodedTiKVKeyPtr = std::shared_ptr; struct RegionQueryInfo { + RegionQueryInfo(RegionID region_id_, UInt64 version_, UInt64 conf_version_, Int64 physical_table_id_, const std::pair & range_in_table_ = {}, const std::vector> & required_handle_ranges_ = {}) + : region_id(region_id_) + , version(version_) + , conf_version(conf_version_) + , physical_table_id(physical_table_id_) + , range_in_table(range_in_table_) + , required_handle_ranges(required_handle_ranges_) + {} RegionID region_id; UInt64 version; UInt64 conf_version; + Int64 physical_table_id; std::pair range_in_table; // required handle ranges is the handle range specified in DAG request std::vector> required_handle_ranges; diff --git a/dbms/src/Storages/Transaction/LearnerRead.cpp b/dbms/src/Storages/Transaction/LearnerRead.cpp index 05b7b7f3ec6..7d2382b63e8 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.cpp +++ b/dbms/src/Storages/Transaction/LearnerRead.cpp @@ -64,13 +64,13 @@ struct UnavailableRegions : public LockWrap doAdd(region_id_); } - void tryThrowRegionException(const MvccQueryInfo::RegionsQueryInfo & regions_info) + void tryThrowRegionException(bool batch_cop) { auto lock = genLockGuard(); // For batch-cop request, all unavailable regions, include the ones with lock exception, should be collected and retry next round. // For normal cop request, which only contains one region, LockException should be thrown directly and let upper layer(like client-c, tidb, tispark) handle it. - if (regions_info.size() == 1 && region_lock) + if (!batch_cop && region_lock) throw LockException(region_lock->first, std::move(region_lock->second)); if (!ids.empty()) @@ -101,7 +101,7 @@ class MvccQueryInfoWrap Base::RegionsQueryInfo * regions_info_ptr; public: - MvccQueryInfoWrap(Base & mvcc_query_info, TMTContext & tmt, const TiDB::TableID table_id) + MvccQueryInfoWrap(Base & mvcc_query_info, TMTContext & tmt, const TiDB::TableID logical_table_id) : inner(mvcc_query_info) { if (likely(!inner.regions_query_info.empty())) @@ -113,14 +113,15 @@ class MvccQueryInfoWrap regions_info = Base::RegionsQueryInfo(); regions_info_ptr = &*regions_info; // Only for test, because regions_query_info should never be empty if query is from TiDB or TiSpark. - auto regions = tmt.getRegionTable().getRegionsByTable(table_id); + // todo support partition table + auto regions = tmt.getRegionTable().getRegionsByTable(logical_table_id); regions_info_ptr->reserve(regions.size()); for (const auto & [id, region] : regions) { if (region == nullptr) continue; regions_info_ptr->emplace_back( - RegionQueryInfo{id, region->version(), region->confVer(), region->getRange()->rawKeys(), {}}); + RegionQueryInfo{id, region->version(), region->confVer(), logical_table_id, region->getRange()->rawKeys(), {}}); } } } @@ -142,10 +143,10 @@ class MvccQueryInfoWrap }; LearnerReadSnapshot doLearnerRead( - const TiDB::TableID table_id, + const TiDB::TableID logical_table_id, MvccQueryInfo & mvcc_query_info_, size_t num_streams, - bool wait_index_timeout_as_region_not_found, + bool for_batch_cop, Context & context, const LogWithPrefixPtr & log) { @@ -153,7 +154,7 @@ LearnerReadSnapshot doLearnerRead( auto & tmt = context.getTMTContext(); - MvccQueryInfoWrap mvcc_query_info(mvcc_query_info_, tmt, table_id); + MvccQueryInfoWrap mvcc_query_info(mvcc_query_info_, tmt, logical_table_id); const auto & regions_info = mvcc_query_info.getRegionsInfo(); // adjust concurrency by num of regions or num of streams * mvcc_query_info.concurrent @@ -296,8 +297,8 @@ LearnerReadSnapshot doLearnerRead( } } - auto handle_wait_timeout_region = [&unavailable_regions, wait_index_timeout_as_region_not_found](const DB::RegionID region_id) { - if (wait_index_timeout_as_region_not_found) + auto handle_wait_timeout_region = [&unavailable_regions, for_batch_cop](const DB::RegionID region_id) { + if (!for_batch_cop) { // If server is being terminated / time-out, add the region_id into `unavailable_regions` to other store. unavailable_regions.add(region_id, RegionException::RegionReadStatus::NOT_FOUND); @@ -310,6 +311,7 @@ LearnerReadSnapshot doLearnerRead( for (size_t region_idx = region_begin_idx, read_index_res_idx = 0; region_idx < region_end_idx; ++region_idx, ++read_index_res_idx) { const auto & region_to_query = regions_info[region_idx]; + Int64 physical_table_id = region_to_query.physical_table_id; // if region is unavailable, skip wait index. if (unavailable_regions.contains(region_to_query.region_id)) @@ -351,7 +353,7 @@ LearnerReadSnapshot doLearnerRead( { auto res = RegionTable::resolveLocksAndWriteRegion( tmt, - table_id, + physical_table_id, region, mvcc_query_info->read_tso, region_to_query.bypass_lock_ts, @@ -406,7 +408,7 @@ LearnerReadSnapshot doLearnerRead( pool.wait(); } - unavailable_regions.tryThrowRegionException(regions_info); + unavailable_regions.tryThrowRegionException(for_batch_cop); auto end_time = Clock::now(); LOG_FMT_DEBUG( diff --git a/dbms/src/Storages/Transaction/LearnerRead.h b/dbms/src/Storages/Transaction/LearnerRead.h index c5611033726..a7c36183d4a 100644 --- a/dbms/src/Storages/Transaction/LearnerRead.h +++ b/dbms/src/Storages/Transaction/LearnerRead.h @@ -29,7 +29,7 @@ doLearnerRead( const TiDB::TableID table_id, MvccQueryInfo & mvcc_query_info, size_t num_streams, - bool wait_index_timeout_as_region_not_found, + bool for_batch_cop, Context & context, const LogWithPrefixPtr & log); diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 59b6ab9053e..2e3bf7748db 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -14,7 +14,7 @@ "type": "grafana", "id": "grafana", "name": "Grafana", - "version": "6.1.6" + "version": "7.5.11" }, { "type": "panel", @@ -52,11 +52,12 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1631518930264, + "iteration": 1647230387563, "links": [], "panels": [ { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -1035,6 +1036,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -2205,6 +2207,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -2637,6 +2640,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -4467,6 +4471,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -4483,7 +4488,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The throughput of write and delta's background management", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 9, "w": 24, @@ -4491,6 +4501,7 @@ "y": 5 }, "height": "", + "hiddenSeries": false, "id": 70, "legend": { "alignAsTable": true, @@ -4512,7 +4523,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -4528,11 +4543,13 @@ "steppedLine": false, "targets": [ { + "exemplar": true, "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m]))", "format": "time_series", "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "throughput_write", + "legendFormat": "throughput_write+ingest", "refId": "A", "step": 10 }, @@ -4544,10 +4561,12 @@ "refId": "B" }, { + "exemplar": true, "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"})", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "total_write", + "legendFormat": "total_write+ingest", "refId": "C" }, { @@ -4606,13 +4625,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The stall duration of write and delete range", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 8, "w": 24, "x": 0, "y": 14 }, + "hiddenSeries": false, "id": 62, "legend": { "alignAsTable": true, @@ -4629,7 +4654,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -4708,7 +4737,12 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "The throughput of write by instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { "h": 9, "w": 24, @@ -4716,6 +4750,7 @@ "y": 22 }, "height": "", + "hiddenSeries": false, "id": 89, "legend": { "alignAsTable": true, @@ -4737,7 +4772,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -4753,13 +4792,23 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m])) by (instance)", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write\"}[1m])) by (instance)", "format": "time_series", "hide": false, + "interval": "", "intervalFactor": 1, "legendFormat": "throughput_write-{{instance}}", "refId": "A", "step": 10 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "throughput_ingest-{{instance}}", + "refId": "B" } ], "thresholds": [], @@ -4810,13 +4859,19 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "The total count of different kinds of commands received", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 0, + "fillGradient": 0, "gridPos": { "h": 9, "w": 24, "x": 0, "y": 31 }, + "hiddenSeries": false, "id": 90, "legend": { "alignAsTable": true, @@ -4833,7 +4888,11 @@ "linewidth": 1, "links": [], "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -4912,6 +4971,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -6222,6 +6282,7 @@ }, { "collapsed": true, + "datasource": null, "gridPos": { "h": 1, "w": 24, @@ -6450,8 +6511,8 @@ "type": "row" } ], - "refresh": "30s", - "schemaVersion": 18, + "refresh": false, + "schemaVersion": 27, "style": "dark", "tags": [], "templating": { @@ -6469,7 +6530,10 @@ "multi": false, "name": "k8s_cluster", "options": [], - "query": "label_values(tiflash_system_profile_event_Query, k8s_cluster)", + "query": { + "query": "label_values(tiflash_system_profile_event_Query, k8s_cluster)", + "refId": "j2-k8s_cluster-Variable-Query" + }, "refresh": 2, "regex": "", "skipUrlSync": false, @@ -6493,7 +6557,10 @@ "multi": false, "name": "tidb_cluster", "options": [], - "query": "label_values(tiflash_system_profile_event_Query{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "query": { + "query": "label_values(tiflash_system_profile_event_Query{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", + "refId": "j2-tidb_cluster-Variable-Query" + }, "refresh": 2, "regex": "", "skipUrlSync": false, @@ -6517,7 +6584,10 @@ "multi": true, "name": "instance", "options": [], - "query": "label_values(tiflash_system_profile_event_Query{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, instance)", + "query": { + "query": "label_values(tiflash_system_profile_event_Query{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, instance)", + "refId": "j2-instance-Variable-Query" + }, "refresh": 1, "regex": "", "skipUrlSync": false, diff --git a/tests/delta-merge-test/query/mpp/partition_table.test b/tests/delta-merge-test/query/mpp/partition_table.test new file mode 100644 index 00000000000..97fabb607c9 --- /dev/null +++ b/tests/delta-merge-test/query/mpp/partition_table.test @@ -0,0 +1,103 @@ +# Preparation. +=> DBGInvoke __enable_schema_sync_service('false') + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> drop table if exists default.test_9997 +=> drop table if exists default.test_9998 +=> drop table if exists default.test_9999 + +=> DBGInvoke __set_flush_threshold(1000000, 1000000) + +# Data. +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64') +=> DBGInvoke __mock_tidb_partition(default, test, 9997) +=> DBGInvoke __mock_tidb_partition(default, test, 9998) +=> DBGInvoke __mock_tidb_partition(default, test, 9999) +=> DBGInvoke __refresh_schemas() + +=> DBGInvoke __put_region(1, 0, 100, default, test, 9997) + +=> DBGInvoke __put_region(4, 0, 100, default, test, 9998) + +=> DBGInvoke __put_region(7, 0, 100, default, test, 9999) + +# query on empty table +=> DBGInvoke tidb_query('select count(col_1) from default.test group by col_2', 4,'mpp_query:true,mpp_partition_num:3') + +=> DBGInvoke __raft_insert_row(default, test, 1, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 1, 51, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 1, 52, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 1, 53, 'test4', 888) + +=> DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 51, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 4, 52, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 4, 53, 'test4', 888) + +=> DBGInvoke __raft_insert_row(default, test, 7, 50, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 7, 51, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 7, 52, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 7, 53, 'test4', 888) + +# query on table that some partition does not contains region +=> DBGInvoke tidb_query('select count(col_1), col_2 from default.test group by col_2', 4,'mpp_query:true,mpp_partition_num:3') +┌─exchange_receiver_0─┬─exchange_receiver_1─┐ +│ 6 │ 666 │ +│ 3 │ 777 │ +│ 3 │ 888 │ +└─────────────────────┴─────────────────────┘ + +# add more regions +=> DBGInvoke __put_region(2, 100, 200, default, test, 9997) +=> DBGInvoke __put_region(3, 200, 300, default, test, 9997) + +=> DBGInvoke __put_region(5, 100, 200, default, test, 9998) +=> DBGInvoke __put_region(6, 200, 300, default, test, 9998) + +=> DBGInvoke __put_region(8, 100, 200, default, test, 9999) +=> DBGInvoke __put_region(9, 200, 300, default, test, 9999) + +=> DBGInvoke __raft_insert_row(default, test, 2, 150, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 2, 151, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 2, 152, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 2, 153, 'test4', 888) +=> DBGInvoke __raft_insert_row(default, test, 3, 250, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 3, 251, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 3, 252, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 3, 253, 'test4', 888) + +=> DBGInvoke __raft_insert_row(default, test, 5, 150, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 5, 151, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 5, 152, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 5, 153, 'test4', 888) +=> DBGInvoke __raft_insert_row(default, test, 6, 250, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 6, 251, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 6, 252, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 6, 253, 'test4', 888) + +=> DBGInvoke __raft_insert_row(default, test, 8, 150, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 8, 151, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 8, 152, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 8, 153, 'test4', 888) +=> DBGInvoke __raft_insert_row(default, test, 9, 250, 'test1', 666) +=> DBGInvoke __raft_insert_row(default, test, 9, 251, 'test2', 666) +=> DBGInvoke __raft_insert_row(default, test, 9, 252, 'test3', 777) +=> DBGInvoke __raft_insert_row(default, test, 9, 253, 'test4', 888) + +# query on table that every partition contains region +=> DBGInvoke tidb_query('select count(col_1), col_2 from default.test group by col_2', 4,'mpp_query:true,mpp_partition_num:3') +┌─exchange_receiver_0─┬─exchange_receiver_1─┐ +│ 18 │ 666 │ +│ 9 │ 777 │ +│ 9 │ 888 │ +└─────────────────────┴─────────────────────┘ + +# Clean up. +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> drop table if exists default.test_9997 +=> drop table if exists default.test_9998 +=> drop table if exists default.test_9999 +=> DBGInvoke __refresh_schemas() +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/fullstack-test/expr/day_of_month.test b/tests/fullstack-test/expr/day_of_month.test new file mode 100644 index 00000000000..3ab2fed9864 --- /dev/null +++ b/tests/fullstack-test/expr/day_of_month.test @@ -0,0 +1,27 @@ +mysql> drop table if exists test.t1; +mysql> create table test.t1 (c_str varchar(100), c_datetime datetime(4), c_date date); +mysql> insert into test.t1 values('', '1999-10-10 10:10:10.123', '1999-01-10'), ('200', '1999-02-10 10:10:10.123', '1999-11-10'), ('1999-30-10', '1999-10-10 10:10:10.123', '1999-01-10'), ('1999-01-10', '1999-10-10 10:10:10.123', '1999-01-10'); +mysql> alter table test.t1 set tiflash replica 1; +func> wait_table test t1 + +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select dayofmonth(); +ERROR 1582 (42000) at line 1: Incorrect parameter count in the call to native function 'dayofmonth' + +# invalid input +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select dayofmonth(''), dayofmonth('1'), dayofmonth('1999-30-01'), dayofmonth(null); ++----------------+-----------------+--------------------------+------------------+ +| dayofmonth('') | dayofmonth('1') | dayofmonth('1999-30-01') | dayofmonth(null) | ++----------------+-----------------+--------------------------+------------------+ +| NULL | NULL | NULL | NULL | ++----------------+-----------------+--------------------------+------------------+ + +# got bug: https://github.com/pingcap/tics/issues/4186 +# mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select dayofmonth(c_str), dayofmonth(c_datetime), dayofmonth(c_date) from test.t1 order by 1, 2, 3; +# +-------------------+------------------------+--------------------+ +# | dayofmonth(c_str) | dayofmonth(c_datetime) | dayofmonth(c_date) | +# +-------------------+------------------------+--------------------+ +# | NULL | 10 | 10 | +# | NULL | 10 | 10 | +# | NULL | 10 | 10 | +# | 10 | 10 | 10 | +# +-------------------+------------------------+--------------------+ diff --git a/tests/fullstack-test/expr/last_day.test b/tests/fullstack-test/expr/last_day.test new file mode 100644 index 00000000000..e4e838a52d6 --- /dev/null +++ b/tests/fullstack-test/expr/last_day.test @@ -0,0 +1,147 @@ +mysql> drop table if exists test.t1; +mysql> create table test.t1(c1 varchar(100), c2 datetime, c3 date); +mysql> insert into test.t1 values('', '1999-10-10 10:10:10.123', '1999-01-10'), ('200', '1999-02-10 10:10:10.123', '1999-11-10'), ('1999-01-10', '1999-10-10 10:10:10.123', '1999-01-10'); +# leap year +mysql> insert into test.t1 values('2000-2-10', '2000-2-10 10:10:10', '2000-2-10'); +# non leap year +mysql> insert into test.t1 values('2001-2-10', '2001-2-10 10:10:10', '2001-2-10'); +# zero day +mysql> insert into test.t1 values('2000-2-0', '2000-2-10 10:10:10', '2000-2-10'); +mysql> alter table test.t1 set tiflash replica 1; +func> wait_table test t1 +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c1, last_day(c1) from test.t1 order by 1; ++------------+--------------+ +| c1 | last_day(c1) | ++------------+--------------+ +| | NULL | +| 1999-01-10 | 1999-01-31 | +| 200 | NULL | +| 2000-2-0 | NULL | +| 2000-2-10 | 2000-02-29 | +| 2001-2-10 | 2001-02-28 | ++------------+--------------+ +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c2, last_day(c2) from test.t1 order by 1; ++---------------------+--------------+ +| c2 | last_day(c2) | ++---------------------+--------------+ +| 1999-02-10 10:10:10 | 1999-02-28 | +| 1999-10-10 10:10:10 | 1999-10-31 | +| 1999-10-10 10:10:10 | 1999-10-31 | +| 2000-02-10 10:10:10 | 2000-02-29 | +| 2000-02-10 10:10:10 | 2000-02-29 | +| 2001-02-10 10:10:10 | 2001-02-28 | ++---------------------+--------------+ +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c3, last_day(c3) from test.t1 order by 1; ++------------+--------------+ +| c3 | last_day(c3) | ++------------+--------------+ +| 1999-01-10 | 1999-01-31 | +| 1999-01-10 | 1999-01-31 | +| 1999-11-10 | 1999-11-30 | +| 2000-02-10 | 2000-02-29 | +| 2000-02-10 | 2000-02-29 | +| 2001-02-10 | 2001-02-28 | ++------------+--------------+ + +mysql> drop table if exists test.t1; +mysql> create table test.t1(c1 date); +mysql> insert into test.t1 values('2001-01-01'),('2001-02-01'),('2001-03-01'),('2001-04-01'),('2001-05-01'),('2001-06-01'),('2001-07-01'),('2001-08-01'),('2001-09-01'),('2001-10-01'),('2001-11-01'),('2001-12-01'); +mysql> insert into test.t1 values('2000-01-01'),('2000-02-01'),('2000-03-01'),('2000-04-01'),('2000-05-01'),('2000-06-01'),('2000-07-01'),('2000-08-01'),('2000-09-01'),('2000-10-01'),('2000-11-01'),('2000-12-01'); +mysql> alter table test.t1 set tiflash replica 1; +func> wait_table test t1 +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c1, last_day(c1) from test.t1 order by 1; ++------------+--------------+ +| c1 | last_day(c1) | ++------------+--------------+ +| 2000-01-01 | 2000-01-31 | +| 2000-02-01 | 2000-02-29 | +| 2000-03-01 | 2000-03-31 | +| 2000-04-01 | 2000-04-30 | +| 2000-05-01 | 2000-05-31 | +| 2000-06-01 | 2000-06-30 | +| 2000-07-01 | 2000-07-31 | +| 2000-08-01 | 2000-08-31 | +| 2000-09-01 | 2000-09-30 | +| 2000-10-01 | 2000-10-31 | +| 2000-11-01 | 2000-11-30 | +| 2000-12-01 | 2000-12-31 | +| 2001-01-01 | 2001-01-31 | +| 2001-02-01 | 2001-02-28 | +| 2001-03-01 | 2001-03-31 | +| 2001-04-01 | 2001-04-30 | +| 2001-05-01 | 2001-05-31 | +| 2001-06-01 | 2001-06-30 | +| 2001-07-01 | 2001-07-31 | +| 2001-08-01 | 2001-08-31 | +| 2001-09-01 | 2001-09-30 | +| 2001-10-01 | 2001-10-31 | +| 2001-11-01 | 2001-11-30 | +| 2001-12-01 | 2001-12-31 | ++------------+--------------+ + +mysql> drop table if exists test.t1; +mysql> create table test.t1(c1 varchar(100)); +mysql> insert into test.t1 values('2001-01-00'),('2001-02-00'),('2001-03-00'),('2001-04-00'),('2001-05-00'),('2001-06-00'),('2001-07-00'),('2001-08-00'),('2001-09-00'),('2001-10-00'),('2001-11-00'),('2001-12-00'); +mysql> insert into test.t1 values('2000-01-00'),('2000-02-00'),('2000-03-00'),('2000-04-00'),('2000-05-00'),('2000-06-00'),('2000-07-00'),('2000-08-00'),('2000-09-00'),('2000-10-00'),('2000-11-00'),('2000-12-00'); +mysql> alter table test.t1 set tiflash replica 1; +func> wait_table test t1 +mysql> set @@sql_mode = 'ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c1, last_day(c1) from test.t1 order by 1; ++------------+--------------+ +| c1 | last_day(c1) | ++------------+--------------+ +| 2000-01-00 | NULL | +| 2000-02-00 | NULL | +| 2000-03-00 | NULL | +| 2000-04-00 | NULL | +| 2000-05-00 | NULL | +| 2000-06-00 | NULL | +| 2000-07-00 | NULL | +| 2000-08-00 | NULL | +| 2000-09-00 | NULL | +| 2000-10-00 | NULL | +| 2000-11-00 | NULL | +| 2000-12-00 | NULL | +| 2001-01-00 | NULL | +| 2001-02-00 | NULL | +| 2001-03-00 | NULL | +| 2001-04-00 | NULL | +| 2001-05-00 | NULL | +| 2001-06-00 | NULL | +| 2001-07-00 | NULL | +| 2001-08-00 | NULL | +| 2001-09-00 | NULL | +| 2001-10-00 | NULL | +| 2001-11-00 | NULL | +| 2001-12-00 | NULL | ++------------+--------------+ +# mysql> set @@sql_mode = ''; +# mysql> set @@tidb_isolation_read_engines='tiflash'; set @@tidb_enforce_mpp = 1; select c1, last_day(c1) from test.t1 order by 1; +# +------------+--------------+ +# | c1 | last_day(c1) | +# +------------+--------------+ +# | 2000-01-00 | 2000-01-31 | +# | 2000-02-00 | 2000-02-29 | +# | 2000-03-00 | 2000-03-31 | +# | 2000-04-00 | 2000-04-30 | +# | 2000-05-00 | 2000-05-31 | +# | 2000-06-00 | 2000-06-30 | +# | 2000-07-00 | 2000-07-31 | +# | 2000-08-00 | 2000-08-31 | +# | 2000-09-00 | 2000-09-30 | +# | 2000-10-00 | 2000-10-31 | +# | 2000-11-00 | 2000-11-30 | +# | 2000-12-00 | 2000-12-31 | +# | 2001-01-00 | 2001-01-31 | +# | 2001-02-00 | 2001-02-28 | +# | 2001-03-00 | 2001-03-31 | +# | 2001-04-00 | 2001-04-30 | +# | 2001-05-00 | 2001-05-31 | +# | 2001-06-00 | 2001-06-30 | +# | 2001-07-00 | 2001-07-31 | +# | 2001-08-00 | 2001-08-31 | +# | 2001-09-00 | 2001-09-30 | +# | 2001-10-00 | 2001-10-31 | +# | 2001-11-00 | 2001-11-30 | +# | 2001-12-00 | 2001-12-31 | +# +------------+--------------+ diff --git a/tests/fullstack-test/mpp/dynamic_partition_prune.test b/tests/fullstack-test/mpp/dynamic_partition_prune.test new file mode 100644 index 00000000000..73ef90e4021 --- /dev/null +++ b/tests/fullstack-test/mpp/dynamic_partition_prune.test @@ -0,0 +1,62 @@ +# Preparation. +=> DBGInvoke __init_fail_point() + +mysql> drop table if exists test.employees; +mysql> CREATE TABLE test.employees ( id int(11) NOT NULL, fname varchar(30) DEFAULT NULL, lname varchar(30) DEFAULT NULL, hired date NOT NULL DEFAULT '1970-01-01', separated date DEFAULT '9999-12-31', job_code int(11) DEFAULT NULL, store_id int(11) NOT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY RANGE (store_id) (PARTITION p0 VALUES LESS THAN (6), PARTITION p1 VALUES LESS THAN (11), PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21)); +mysql> alter table test.employees set tiflash replica 1; +mysql> insert into test.employees values(100,'aa','aa','2020-01-01',null,10,5); +mysql> insert into test.employees values(100,'aa','aa','2020-01-01',null,10,15); +mysql> insert into test.employees values(100,'aa','aa','2020-01-01',null,10,20); + +func> wait_table test employees + +# no partition prune +mysql> set tidb_partition_prune_mode=dynamic; set tidb_enforce_mpp=1; select count(*) from test.employees; ++----------+ +| count(*) | ++----------+ +| 3 | ++----------+ + +# no partition after prune +mysql> set tidb_partition_prune_mode=dynamic; set tidb_enforce_mpp=1; select count(*) from test.employees where store_id > 100; ++----------+ +| count(*) | ++----------+ +| 0 | ++----------+ + +# 1 partition left after prune +mysql> set tidb_partition_prune_mode=dynamic; set tidb_enforce_mpp=1; select count(*) from test.employees where store_id < 6; ++----------+ +| count(*) | ++----------+ +| 1 | ++----------+ + +# 1 partition left after prune +mysql> set tidb_partition_prune_mode=dynamic; set tidb_enforce_mpp=1; select count(*) from test.employees where store_id > 6 and store_id < 11; ++----------+ +| count(*) | ++----------+ +| 0 | ++----------+ + +# 2 partition left after prune +mysql> set tidb_partition_prune_mode=dynamic; set tidb_enforce_mpp=1; select count(*) from test.employees where store_id < 11; ++----------+ +| count(*) | ++----------+ +| 1 | ++----------+ + +# all partitions left after prune +mysql> set tidb_partition_prune_mode=dynamic; set tidb_enforce_mpp=1; select count(*) from test.employees where store_id < 100; ++----------+ +| count(*) | ++----------+ +| 3 | ++----------+ + +# Clean up. +mysql> drop table if exists test.employees;