diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 747ff1d91cc..923505d93fa 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -26,10 +26,12 @@ Tests Side effects -# - Performance regression -# - Consumes more CPU -# - Consumes more MEM -# - Breaking backward compatibility + ### Release note diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 12da0dde3b7..548d83cec02 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -390,6 +390,7 @@ namespace ErrorCodes extern const int PAGE_SIZE_NOT_MATCH = 9006; extern const int ILLFORMED_PAGE_NAME = 9007; extern const int ILLFORMAT_RAFT_ROW = 9008; + extern const int REGION_DATA_SCHEMA_UPDATED = 9009; extern const int LOCK_EXCEPTION = 10000; extern const int VERSION_ERROR = 10001; diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 51661bd7c6a..bd7bea5b2b5 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -37,7 +37,9 @@ std::unordered_map> FailPointHelper::f M(exception_before_mpp_register_tunnel_for_root_mpp_task) \ M(exception_before_mpp_root_task_run) \ M(exception_during_mpp_root_task_run) \ - M(exception_during_write_to_storage) + M(exception_during_write_to_storage) \ + M(force_set_sst_to_dtfile_block_size) \ + M(force_set_sst_decode_rand) #define APPLY_FOR_FAILPOINTS_WITH_CHANNEL(M) \ M(pause_after_learner_read) \ diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index d30e818d5f2..1a10d8f9ac0 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -84,7 +84,7 @@ namespace DB M(tiflash_storage_write_amplification, "The data write amplification in storage engine", Gauge) \ M(tiflash_storage_read_tasks_count, "Total number of storage engine read tasks", Counter) \ M(tiflash_storage_command_count, "Total number of storage's command, such as delete range / shutdown /startup", Counter, \ - F(type_delete_range, {"type", "delete_range"})) \ + F(type_delete_range, {"type", "delete_range"}), F(type_ingest, {"type", "ingest"})) \ M(tiflash_storage_subtask_count, "Total number of storage's sub task", Counter, F(type_delta_merge, {"type", "delta_merge"}), \ F(type_delta_merge_fg, {"type", "delta_merge_fg"}), F(type_delta_merge_bg_gc, {"type", "delta_merge_bg_gc"}), \ F(type_delta_compact, {"type", "delta_compact"}), F(type_delta_flush, {"type", "delta_flush"}), \ @@ -101,11 +101,13 @@ namespace DB F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.0005, 2, 20})) \ M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ + F(type_ingest, {"type", "ingest"}), /**/ \ F(type_delta_merge, {"type", "delta_merge"}), /**/ \ F(type_split, {"type", "split"}), /**/ \ F(type_merge, {"type", "merge"})) /**/ \ M(tiflash_storage_throughput_rows, "Calculate the throughput of tasks of storage in rows", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ + F(type_ingest, {"type", "ingest"}), /**/ \ F(type_delta_merge, {"type", "delta_merge"}), /**/ \ F(type_split, {"type", "split"}), /**/ \ F(type_merge, {"type", "merge"})) /**/ \ diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp index 2eb8fec6762..94f9df88dc9 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include diff --git a/dbms/src/DataStreams/BlocksListBlockInputStream.h b/dbms/src/DataStreams/BlocksListBlockInputStream.h index df7feaacda2..22751ac22de 100644 --- a/dbms/src/DataStreams/BlocksListBlockInputStream.h +++ b/dbms/src/DataStreams/BlocksListBlockInputStream.h @@ -17,7 +17,7 @@ class BlocksListBlockInputStream : public IProfilingBlockInputStream : list(std::move(list_)), it(list.begin()), end(list.end()) {} /// Uses a list of blocks lying somewhere else. - BlocksListBlockInputStream(BlocksList::iterator & begin_, BlocksList::iterator & end_) + BlocksListBlockInputStream(const BlocksList::iterator & begin_, const BlocksList::iterator & end_) : it(begin_), end(end_) {} String getName() const override { return "BlocksList"; } @@ -27,7 +27,7 @@ class BlocksListBlockInputStream : public IProfilingBlockInputStream Block res; if (!list.empty()) for (const auto & elem : list.front()) - res.insert({ elem.column->cloneEmpty(), elem.type, elem.name }); + res.insert({elem.column->cloneEmpty(), elem.type, elem.name, elem.column_id}); return res; } diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 50d0a535976..9cb67d84456 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -126,7 +126,7 @@ class IBlockInputStream : private boost::noncopyable protected: BlockInputStreams children; - std::shared_mutex children_mutex; + mutable std::shared_mutex children_mutex; private: TableStructureReadLocks table_locks; diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index a21e1197028..51f242fa2ab 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp b/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp index 53a82de0a53..715e18b419f 100644 --- a/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp +++ b/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp @@ -6,11 +6,6 @@ #include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wsign-compare" -#include -#pragma GCC diagnostic pop - namespace DB { namespace tests diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index 57f9bff2dc2..3594764513e 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -85,8 +85,10 @@ DBGInvoker::DBGInvoker() regSchemalessFunc("region_snapshot", MockRaftCommand::dbgFuncRegionSnapshot); regSchemalessFunc("region_snapshot_data", MockRaftCommand::dbgFuncRegionSnapshotWithData); - regSchemalessFunc("region_snapshot_pre_handle_block", MockRaftCommand::dbgFuncRegionSnapshotPreHandleBlock); - regSchemalessFunc("region_snapshot_apply_block", MockRaftCommand::dbgFuncRegionSnapshotApplyBlock); + regSchemalessFunc("region_snapshot_pre_handle_block", /**/ MockRaftCommand::dbgFuncRegionSnapshotPreHandleBlock); + regSchemalessFunc("region_snapshot_apply_block", /* */ MockRaftCommand::dbgFuncRegionSnapshotApplyBlock); + regSchemalessFunc("region_snapshot_pre_handle_file", /* */ MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFiles); + regSchemalessFunc("region_snapshot_apply_file", /* */ MockRaftCommand::dbgFuncRegionSnapshotApplyDTFiles); regSchemalessFunc("region_ingest_sst", MockRaftCommand::dbgFuncIngestSST); regSchemalessFunc("init_fail_point", DbgFailPointFunc::dbgInitFailPoint); diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index 8f519135f78..84f220ae2f4 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -59,7 +59,12 @@ static const String MPP_QUERY = "mpp_query"; static const String USE_BROADCAST_JOIN = "use_broadcast_join"; static const String MPP_PARTITION_NUM = "mpp_partition_num"; static const String MPP_TIMEOUT = "mpp_timeout"; -static const String LOCAL_HOST = "127.0.0.1:3930"; +static String LOCAL_HOST = "127.0.0.1:3930"; + +namespace Debug +{ +void setServiceAddr(const std::string & addr) { LOCAL_HOST = addr; } +} // namespace Debug struct DAGProperties { diff --git a/dbms/src/Debug/dbgFuncCoprocessor.h b/dbms/src/Debug/dbgFuncCoprocessor.h index 6781f0254a8..e04a280df16 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.h +++ b/dbms/src/Debug/dbgFuncCoprocessor.h @@ -20,4 +20,9 @@ BlockInputStreamPtr dbgFuncTiDBQuery(Context & context, const ASTs & args); // ./storages-client.sh "DBGInvoke mock_dag(query, region_id[, start_ts])" BlockInputStreamPtr dbgFuncMockTiDBQuery(Context & context, const ASTs & args); +namespace Debug +{ +void setServiceAddr(const std::string & addr); +} + } // namespace DB diff --git a/dbms/src/Debug/dbgFuncMockRaftCommand.h b/dbms/src/Debug/dbgFuncMockRaftCommand.h index 27912cecf4d..7a3dd8aa1d2 100644 --- a/dbms/src/Debug/dbgFuncMockRaftCommand.h +++ b/dbms/src/Debug/dbgFuncMockRaftCommand.h @@ -57,6 +57,16 @@ struct MockRaftCommand // Usage: // ./storages-client.sh "DBGInvoke region_snapshot_apply_block(region_id)" static void dbgFuncRegionSnapshotApplyBlock(Context & context, const ASTs & args, DBGInvoker::Printer output); + + // Simulate a region pre-handle snapshot data to DTFiles + // Usage: + // ./storage-client.sh "DBGInvoke region_snapshot_pre_handle_file(database_name, table_name, region_id, start, end, schema_string, pk_name[, test-fields=1, cfs="write,default"])" + static void dbgFuncRegionSnapshotPreHandleDTFiles(Context & context, const ASTs & args, DBGInvoker::Printer output); + + // Apply snapshot for a region. (apply a pre-handle snapshot) + // Usage: + // ./storages-client.sh "DBGInvoke region_snapshot_apply_file(region_id)" + static void dbgFuncRegionSnapshotApplyDTFiles(Context & context, const ASTs & args, DBGInvoker::Printer output); }; } // namespace DB diff --git a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp index 61fe36801f9..1757720621b 100644 --- a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp +++ b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp @@ -29,8 +29,9 @@ namespace DB namespace FailPoints { -extern const char force_set_prehandle_dtfile_block_size[]; -} +extern const char force_set_sst_to_dtfile_block_size[]; +extern const char force_set_sst_decode_rand[]; +} // namespace FailPoints namespace ErrorCodes { @@ -143,7 +144,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotWithData(Context & context, const AST // Mock to apply a snapshot with data in `region` auto & tmt = context.getTMTContext(); - context.getTMTContext().getKVStore()->checkAndApplySnapshot(region, tmt); + context.getTMTContext().getKVStore()->checkAndApplySnapshot(region, tmt); std::stringstream ss; ss << "put region #" << region_id << ", range" << range_string << " to table #" << table_id << " with " << cnt << " records"; output(ss.str()); @@ -402,6 +403,7 @@ void MockRaftCommand::dbgFuncIngestSST(Context & context, const ASTs & args, DBG auto & kvstore = tmt.getKVStore(); auto region = kvstore->getRegion(region_id); + FailPointHelper::enableFailPoint(FailPoints::force_set_sst_decode_rand); // Register some mock SST reading methods so that we can decode data in `MockSSTReader::MockSSTData` RegionMockTest mock_test(kvstore, region); @@ -517,7 +519,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotApplyBlock(Context & context, const A RegionID region_id = (RegionID)safeGet(typeid_cast(*args.front()).value); auto [region, block_cache] = GLOBAL_REGION_MAP.popRegionCache("__snap_" + std::to_string(region_id)); auto & tmt = context.getTMTContext(); - context.getTMTContext().getKVStore()->checkAndApplySnapshot({region, std::move(block_cache)}, tmt); + context.getTMTContext().getKVStore()->checkAndApplySnapshot({region, std::move(block_cache)}, tmt); std::stringstream ss; ss << "success apply " << region->id() << " with block cache"; @@ -525,4 +527,121 @@ void MockRaftCommand::dbgFuncRegionSnapshotApplyBlock(Context & context, const A } +/// Mock to pre-decode snapshot to DTFile(s) then apply + +// Simulate a region pre-handle snapshot data to DTFiles +// ./storage-client.sh "DBGInvoke region_snapshot_pre_handle_file(database_name, table_name, region_id, start, end, schema_string, pk_name[, test-fields=1, cfs="write,default"])" +void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFiles(Context & context, const ASTs & args, DBGInvoker::Printer output) +{ + if (args.size() < 7 || args.size() > 9) + throw Exception("Args not matched, should be: database_name, table_name, region_id, start, end, schema_string, pk_name" + " [, test-fields, cfs=\"write,default\"]", + ErrorCodes::BAD_ARGUMENTS); + + const String & database_name = typeid_cast(*args[0]).name; + const String & table_name = typeid_cast(*args[1]).name; + RegionID region_id = (RegionID)safeGet(typeid_cast(*args[2]).value); + RegionID start_handle = (RegionID)safeGet(typeid_cast(*args[3]).value); + RegionID end_handle = (RegionID)safeGet(typeid_cast(*args[4]).value); + + const String schema_str = safeGet(typeid_cast(*args[5]).value); + String handle_pk_name = safeGet(typeid_cast(*args[6]).value); + + UInt64 test_fields = 1; + if (args.size() > 7) + test_fields = (UInt64)safeGet(typeid_cast(*args[7]).value); + std::unordered_set cfs; + { + String cfs_str = "write,default"; + if (args.size() > 8) + cfs_str = safeGet(typeid_cast(*args[8]).value); + if (cfs_str.find("write") != std::string::npos) + cfs.insert(ColumnFamilyType::Write); + if (cfs_str.find("default") != std::string::npos) + cfs.insert(ColumnFamilyType::Default); + } + + // Parse a TableInfo from `schema_str` to generate data with this schema + TiDB::TableInfoPtr mocked_table_info; + { + ASTPtr columns_ast; + ParserColumnDeclarationList schema_parser; + Tokens tokens(schema_str.data(), schema_str.data() + schema_str.length()); + TokenIterator pos(tokens); + Expected expected; + if (!schema_parser.parse(pos, columns_ast, expected)) + throw Exception("Invalid TiDB table schema", ErrorCodes::LOGICAL_ERROR); + ColumnsDescription columns + = InterpreterCreateQuery::getColumnsDescription(typeid_cast(*columns_ast), context); + mocked_table_info = MockTiDB::parseColumns(table_name, columns, handle_pk_name, "dt"); + } + + MockTiDB::TablePtr table = MockTiDB::instance().getTableByName(database_name, table_name); + const auto & table_info = RegionBench::getTableInfo(context, database_name, table_name); + if (table_info.is_common_handle) + throw Exception("Mocking pre handle SST files to DTFiles to a common handle table is not supported", ErrorCodes::LOGICAL_ERROR); + + // Mock SST data for handle [start, end) + const auto region_name = "__snap_snap_" + std::to_string(region_id); + GenMockSSTData(*mocked_table_info, table->id(), region_name, start_handle, end_handle, test_fields, cfs); + + auto & tmt = context.getTMTContext(); + auto & kvstore = tmt.getKVStore(); + auto old_region = kvstore->getRegion(region_id); + + // We may call this function mutiple time to mock some situation, try to reuse the region in `GLOBAL_REGION_MAP` + // so that we can collect uncommitted data. + UInt64 index = MockTiKV::instance().getRaftIndex(region_id) + 1; + RegionPtr new_region = RegionBench::createRegion(table->id(), region_id, start_handle, end_handle, index); + + // Register some mock SST reading methods so that we can decode data in `MockSSTReader::MockSSTData` + RegionMockTest mock_test(kvstore, new_region); + + std::vector sst_views; + { + if (cfs.count(ColumnFamilyType::Write) > 0) + sst_views.push_back(SSTView{ + ColumnFamilyType::Write, + BaseBuffView{region_name.data(), region_name.length()}, + }); + if (cfs.count(ColumnFamilyType::Default) > 0) + sst_views.push_back(SSTView{ + ColumnFamilyType::Default, + BaseBuffView{region_name.data(), region_name.length()}, + }); + } + + // set block size so that we can test for schema-sync while decoding dt files + FailPointHelper::enableFailPoint(FailPoints::force_set_sst_to_dtfile_block_size); + + auto ingest_ids = kvstore->preHandleSnapshotToFiles( + new_region, SSTViewVec{sst_views.data(), sst_views.size()}, index, MockTiKV::instance().getRaftTerm(region_id), tmt); + GLOBAL_REGION_MAP.insertRegionSnap(region_name, {new_region, ingest_ids}); + + { + std::stringstream ss; + ss << "Generate " << ingest_ids.size() << " files for [region_id=" << region_id << "]"; + output(ss.str()); + } +} + +// Apply snapshot for a region. (apply a pre-handle snapshot) +// ./storages-client.sh "DBGInvoke region_snapshot_apply_file(region_id)" +void MockRaftCommand::dbgFuncRegionSnapshotApplyDTFiles(Context & context, const ASTs & args, DBGInvoker::Printer output) +{ + if (args.size() != 1) + throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS); + + RegionID region_id = (RegionID)safeGet(typeid_cast(*args.front()).value); + const auto region_name = "__snap_snap_" + std::to_string(region_id); + auto [new_region, ingest_ids] = GLOBAL_REGION_MAP.popRegionSnap(region_name); + auto & tmt = context.getTMTContext(); + context.getTMTContext().getKVStore()->checkAndApplySnapshot( + RegionPtrWithSnapshotFiles{new_region, std::move(ingest_ids)}, tmt); + + std::stringstream ss; + ss << "success apply region " << new_region->id() << " with dt files"; + output(ss.str()); +} + } // namespace DB diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.h b/dbms/src/Debug/dbgFuncMockTiDBTable.h index 1ffa26197be..ec698461235 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.h +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.h @@ -14,8 +14,8 @@ struct MockTiDBTable // Inject mocked TiDB table. // Usage: - // ./storages-client.sh "DBGInvoke mock_tidb_table(database_name, table_name, 'col1 type1, col2 type2, ...' [, handle_pk_name, engine-type(tmt|dt)])" + // ./storages-client.sh "DBGInvoke mock_tidb_table(database_name, table_name, 'col1 type1, col2 type2, ...' [, handle_pk_name, engine-type(tmt|dt)])" // engine: [tmt, dt], tmt by default static void dbgFuncMockTiDBTable(Context & context, const ASTs & args, DBGInvoker::Printer output); diff --git a/dbms/src/Debug/dbgFuncRegion.cpp b/dbms/src/Debug/dbgFuncRegion.cpp index 06c21a4d90f..3b3d062f45a 100644 --- a/dbms/src/Debug/dbgFuncRegion.cpp +++ b/dbms/src/Debug/dbgFuncRegion.cpp @@ -60,7 +60,7 @@ void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer TMTContext & tmt = context.getTMTContext(); RegionPtr region = RegionBench::createRegion(table_info, region_id, start_keys, end_keys); - tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); + tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); std::stringstream ss; ss << "put region #" << region_id << ", range" << RecordKVFormat::DecodedTiKVKeyRangeToDebugString(region->getRange()->rawKeys()) @@ -74,7 +74,7 @@ void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer TMTContext & tmt = context.getTMTContext(); RegionPtr region = RegionBench::createRegion(table_id, region_id, start, end); - tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); + tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); std::stringstream ss; ss << "put region #" << region_id << ", range[" << start << ", " << end << ")" diff --git a/dbms/src/Debug/dbgTools.cpp b/dbms/src/Debug/dbgTools.cpp index 1f6c568df4c..ad0e1e82e51 100644 --- a/dbms/src/Debug/dbgTools.cpp +++ b/dbms/src/Debug/dbgTools.cpp @@ -493,7 +493,7 @@ void concurrentBatchInsert(const TiDB::TableInfo & table_info, Int64 concurrent_ Regions regions = createRegions(table_info.id, concurrent_num, key_num_each_region, handle_begin, curr_max_region_id + 1); for (const RegionPtr & region : regions) - tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); + tmt.getKVStore()->onSnapshot(region, nullptr, 0, tmt); std::list threads; for (Int64 i = 0; i < concurrent_num; i++, handle_begin += key_num_each_region) diff --git a/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp b/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp index fe4c1c820be..b050cc262fb 100644 --- a/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp +++ b/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #pragma GCC diagnostic pop diff --git a/dbms/src/Functions/tests/gtest_datetime_extract.cpp b/dbms/src/Functions/tests/gtest_datetime_extract.cpp index 16b6b1bf39a..f9cb9e79f44 100644 --- a/dbms/src/Functions/tests/gtest_datetime_extract.cpp +++ b/dbms/src/Functions/tests/gtest_datetime_extract.cpp @@ -13,7 +13,6 @@ #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" #include -#include #pragma GCC diagnostic pop diff --git a/dbms/src/Functions/tests/gtest_strings_pad.cpp b/dbms/src/Functions/tests/gtest_strings_pad.cpp index d6b2cf68542..80cc2e6983a 100644 --- a/dbms/src/Functions/tests/gtest_strings_pad.cpp +++ b/dbms/src/Functions/tests/gtest_strings_pad.cpp @@ -13,7 +13,6 @@ #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" #include -#include #pragma GCC diagnostic pop diff --git a/dbms/src/Functions/tests/gtest_strings_trim.cpp b/dbms/src/Functions/tests/gtest_strings_trim.cpp index 4db6bba84f2..3a5c86ab53a 100644 --- a/dbms/src/Functions/tests/gtest_strings_trim.cpp +++ b/dbms/src/Functions/tests/gtest_strings_trim.cpp @@ -11,7 +11,6 @@ #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" #include -#include #pragma GCC diagnostic pop diff --git a/dbms/src/Server/RaftConfigParser.cpp b/dbms/src/Server/RaftConfigParser.cpp index 91fcfa58017..4800e50e705 100644 --- a/dbms/src/Server/RaftConfigParser.cpp +++ b/dbms/src/Server/RaftConfigParser.cpp @@ -68,7 +68,6 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::LayeredConfigurat else res.engine = DEFAULT_ENGINE; } - LOG_DEBUG(log, "Default storage engine: " << static_cast(res.engine)); /// "tmt" engine ONLY support disable_bg_flush = false. /// "dt" engine ONLY support disable_bg_flush = true. @@ -97,6 +96,50 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::LayeredConfigurat res.enable_compatible_mode = config.getBool("raft.enable_compatible_mode"); } + if (config.has("raft.snapshot.method")) + { + String snapshot_method = config.getString("raft.snapshot.method", "file1"); + std::transform(snapshot_method.begin(), snapshot_method.end(), snapshot_method.begin(), [](char ch) { return std::tolower(ch); }); + if (snapshot_method == "block") + { + res.snapshot_apply_method = TiDB::SnapshotApplyMethod::Block; + } + else if (snapshot_method == "file1") + { + res.snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory; + } +#if 0 + else if (snapshot_method == "file2") + { + res.snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Single; + } +#endif + else + { + throw Exception("Illegal arguments: unknown snapshot apply method: " + snapshot_method, ErrorCodes::INVALID_CONFIG_PARAMETER); + } + } + switch (res.snapshot_apply_method) + { + case TiDB::SnapshotApplyMethod::DTFile_Directory: + case TiDB::SnapshotApplyMethod::DTFile_Single: + if (res.engine != TiDB::StorageEngine::DT) + { + throw Exception( + "Illegal arguments: can not use DTFile to store snapshot data when the storage engine is not DeltaTree, [engine=" + + DB::toString(static_cast(res.engine)) + + "] [snapshot method=" + applyMethodToString(res.snapshot_apply_method) + "]", + ErrorCodes::INVALID_CONFIG_PARAMETER); + } + break; + default: + break; + } + + LOG_INFO(log, + "Default storage engine [type=" << static_cast(res.engine) + << "] [snapshot.method=" << applyMethodToString(res.snapshot_apply_method) << "]"); + return res; } diff --git a/dbms/src/Server/RaftConfigParser.h b/dbms/src/Server/RaftConfigParser.h index e74c5248f79..750e83fd180 100644 --- a/dbms/src/Server/RaftConfigParser.h +++ b/dbms/src/Server/RaftConfigParser.h @@ -28,11 +28,13 @@ struct TiFlashRaftConfig static constexpr TiDB::StorageEngine DEFAULT_ENGINE = TiDB::StorageEngine::DT; bool disable_bg_flush = false; TiDB::StorageEngine engine = DEFAULT_ENGINE; + TiDB::SnapshotApplyMethod snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory; public: TiFlashRaftConfig() = default; static TiFlashRaftConfig parseSettings(Poco::Util::LayeredConfiguration & config, Poco::Logger * log); + }; } // namespace DB diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 9b090cad487..39a7f2d8079 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -100,6 +100,10 @@ extern const int ARGUMENT_OUT_OF_BOUND; extern const int INVALID_CONFIG_PARAMETER; } // namespace ErrorCodes +namespace Debug +{ +extern void setServiceAddr(const std::string & addr); +} static std::string getCanonicalPath(std::string path) { @@ -837,6 +841,7 @@ int Server::main(const std::vector & /*args*/) builder.SetMaxSendMessageSize(-1); flash_grpc_server = builder.BuildAndStart(); LOG_INFO(log, "Flash grpc server listening on [" << raft_config.flash_server_addr << "]"); + Debug::setServiceAddr(raft_config.flash_server_addr); } SCOPE_EXIT({ diff --git a/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.cpp index 56648a0829e..957c4e028fb 100644 --- a/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.cpp +++ b/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.cpp @@ -10,6 +10,23 @@ namespace DB namespace DM { +template +void DMVersionFilterBlockInputStream::readPrefix() +{ + forEachChild([](IBlockInputStream & child) { + child.readPrefix(); + return false; + }); +} + +template +void DMVersionFilterBlockInputStream::readSuffix() +{ + forEachChild([](IBlockInputStream & child) { + child.readSuffix(); + return false; + }); +} static constexpr size_t UNROLL_BATCH = 64; @@ -365,4 +382,4 @@ template class DMVersionFilterBlockInputStream; template class DMVersionFilterBlockInputStream; } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.h b/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.h index bdf0fcfd638..1c14fe55553 100644 --- a/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.h +++ b/dbms/src/Storages/DeltaMerge/DMVersionFilterBlockInputStream.h @@ -52,6 +52,9 @@ class DMVersionFilterBlockInputStream : public IBlockInputStream << "%, effective: " << DB::toString((Float64)effective_num_rows * 100 / passed_rows, 2) << "%"); } + void readPrefix() override; + void readSuffix() override; + String getName() const override { return "DeltaMergeVersionFilter"; } Block getHeader() const override { return header; } diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.cpp index c566c293b6c..7617a200bb5 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.cpp @@ -8,11 +8,6 @@ #include #include -namespace ProfileEvents -{ -extern const Event DMWriteBytes; -} - namespace DB { namespace DM @@ -201,7 +196,7 @@ String packsToString(const DeltaPacks & packs) packs_info += "F_" + DB::toString(p->getRows()); else if (auto dp_delete = p->tryToDeleteRange(); dp_delete) packs_info += "D_" + dp_delete->getDeleteRange().toString(); - packs_info += + (p->isSaved() ? "_S," : "_N,"); + packs_info += (p->isSaved() ? "_S," : "_N,"); } if (!packs.empty()) packs_info.erase(packs_info.size() - 1); @@ -210,4 +205,4 @@ String packsToString(const DeltaPacks & packs) } } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.h index 5dde7855ef0..d349e0a9679 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPack.h @@ -154,4 +154,4 @@ DeltaPacks deserializePacks_V3(DMContext & context, const RowKeyRange & segment_ String packsToString(const DeltaPacks & packs); } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.cpp index 0af87c2976a..fa633c612cf 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.cpp @@ -235,7 +235,7 @@ void DeltaPackBlock::serializeMetadata(WriteBuffer & buf, bool save_schema) cons writeIntBinary(bytes, buf); } -DeltaPackPtr DeltaPackBlock::deserializeMetadata(ReadBuffer & buf, const BlockPtr & last_schema) +std::tuple DeltaPackBlock::deserializeMetadata(ReadBuffer & buf, const BlockPtr & last_schema) { auto schema = deserializeSchema(buf); if (!schema) @@ -250,7 +250,7 @@ DeltaPackPtr DeltaPackBlock::deserializeMetadata(ReadBuffer & buf, const BlockPt readIntBinary(rows, buf); readIntBinary(bytes, buf); - return createPackWithDataPage(schema, rows, bytes, data_page_id); + return {createPackWithDataPage(schema, rows, bytes, data_page_id), std::move(schema)}; } ColumnPtr DPBlockReader::getPKColumn() @@ -294,4 +294,4 @@ DeltaPackReaderPtr DPBlockReader::createNewReader(const ColumnDefinesPtr & new_c } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.h index 9ade1a3ebcf..b6364e5dff5 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackBlock.h @@ -118,7 +118,7 @@ class DeltaPackBlock : public DeltaPack void serializeMetadata(WriteBuffer & buf, bool save_schema) const override; - static DeltaPackPtr deserializeMetadata(ReadBuffer & buf, const BlockPtr & last_schema); + static std::tuple deserializeMetadata(ReadBuffer & buf, const BlockPtr & last_schema); String toString() const override { diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp index aac0481569c..8470ad17bf9 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.cpp @@ -42,8 +42,7 @@ void DeltaPackFile::serializeMetadata(WriteBuffer & buf, bool /*save_schema*/) c DeltaPackPtr DeltaPackFile::deserializeMetadata(DMContext & context, // const RowKeyRange & segment_range, - ReadBuffer & buf, - const BlockPtr & /*last_schema*/) + ReadBuffer & buf) { UInt64 file_ref_id; size_t valid_rows, valid_bytes; @@ -153,28 +152,34 @@ size_t DPFileReader::readRowsOnce(MutableColumns & output_cols, // size_t rows_end = rows_offset + rows_limit; size_t actual_read = 0; - while (rows_offset < rows_end) + size_t read_offset = rows_offset; + while (read_offset < rows_end) { if (!cur_block || cur_block_offset == cur_block.rows()) { - if (!read_next_block()) - throw Exception("Not enough delta data to read", ErrorCodes::LOGICAL_ERROR); + if (unlikely(!read_next_block())) + throw Exception("Not enough delta data to read [offset=" + DB::toString(rows_offset) + + "] [limit=" + DB::toString(rows_limit) + "] [read_offset=" + DB::toString(read_offset) + "]", + ErrorCodes::LOGICAL_ERROR); } - if (rows_offset < rows_before_cur_block + cur_block_offset) - throw Exception("rows_offset is too small", ErrorCodes::LOGICAL_ERROR); + if (unlikely(read_offset < rows_before_cur_block + cur_block_offset)) + throw Exception("read_offset is too small [offset=" + DB::toString(rows_offset) + "] [limit=" + DB::toString(rows_limit) + + "] [read_offset=" + DB::toString(read_offset) + + "] [min_offset=" + DB::toString(rows_before_cur_block + cur_block_offset) + "]", + ErrorCodes::LOGICAL_ERROR); - if (rows_offset >= rows_before_cur_block + cur_block.rows()) + if (read_offset >= rows_before_cur_block + cur_block.rows()) { cur_block_offset = cur_block.rows(); continue; } auto read_end_for_cur_block = std::min(rows_end, rows_before_cur_block + cur_block.rows()); - auto read_start_in_block = rows_offset - rows_before_cur_block; - auto read_limit_in_block = read_end_for_cur_block - rows_offset; + auto read_start_in_block = read_offset - rows_before_cur_block; + auto read_limit_in_block = read_end_for_cur_block - read_offset; actual_read += copyColumnsData(cur_block_data, cur_block_data[0], output_cols, read_start_in_block, read_limit_in_block, range); - rows_offset += read_limit_in_block; + read_offset += read_limit_in_block; cur_block_offset += read_limit_in_block; } return actual_read; @@ -184,10 +189,18 @@ size_t DPFileReader::readRows(MutableColumns & output_cols, size_t rows_offset, { initStream(); - if (pk_ver_only) - return readRowsRepeatedly(output_cols, rows_offset, rows_limit, range); - else - return readRowsOnce(output_cols, rows_offset, rows_limit, range); + try + { + if (pk_ver_only) + return readRowsRepeatedly(output_cols, rows_offset, rows_limit, range); + else + return readRowsOnce(output_cols, rows_offset, rows_limit, range); + } + catch (DB::Exception & e) + { + e.addMessage(" while reading DTFile " + pack.getFile()->path()); + throw; + } } Block DPFileReader::readNextBlock() @@ -204,4 +217,4 @@ DeltaPackReaderPtr DPFileReader::createNewReader(const ColumnDefinesPtr & new_co } } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.h index 44128b0c034..a5cf79cfae2 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPackFile.h @@ -65,8 +65,7 @@ class DeltaPackFile : public DeltaPack static DeltaPackPtr deserializeMetadata(DMContext & context, // const RowKeyRange & segment_range, - ReadBuffer & buf, - const BlockPtr & last_schema); + ReadBuffer & buf); String toString() const override { diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaPack_V3.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaPack_V3.cpp index 22ca2105d7b..2b3f3d3807b 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaPack_V3.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaPack_V3.cpp @@ -66,19 +66,17 @@ DeltaPacks deserializePacks_V3(DMContext & context, const RowKeyRange & segment_ pack = DeltaPackDeleteRange::deserializeMetadata(buf); break; case DeltaPack::Type::BLOCK: { - pack = DeltaPackBlock::deserializeMetadata(buf, last_schema); - if (auto dp_block = pack->tryToBlock(); dp_block) - last_schema = dp_block->getSchema(); + std::tie(pack, last_schema) = DeltaPackBlock::deserializeMetadata(buf, last_schema); break; } case DeltaPack::Type::FILE: { - pack = DeltaPackFile::deserializeMetadata(context, segment_range, buf, last_schema); + pack = DeltaPackFile::deserializeMetadata(context, segment_range, buf); break; } default: throw Exception("Unexpected pack type: " + DB::toString(pack_type), ErrorCodes::LOGICAL_ERROR); } - packs.push_back(pack); + packs.emplace_back(std::move(pack)); } return packs; } diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index b0d49ae3825..7806c548917 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -237,7 +237,7 @@ void DeltaValueSpace::appendPackInner(const DeltaPackPtr & pack) // If this pack's schema is identical to last_schema, then use the last_schema instance, // so that we don't have to serialize my_schema instance. auto my_schema = dp_block->getSchema(); - if (last_schema && my_schema && last_schema != my_schema && checkSchema(*my_schema, *last_schema)) + if (last_schema && my_schema && last_schema != my_schema && isSameSchema(*my_schema, *last_schema)) dp_block->resetIdenticalSchema(last_schema); } @@ -289,14 +289,14 @@ bool DeltaValueSpace::appendToCache(DMContext & context, const Block & block, si { if constexpr (DM_RUN_CHECK) { - if (unlikely(!checkSchema(*p->getSchema(), p->getCache()->block))) + if (unlikely(!isSameSchema(*p->getSchema(), p->getCache()->block))) throw Exception("Mutable pack's structure of schema and block are different: " + last_pack->toString()); } auto & cache_block = p->getCache()->block; bool is_overflow = cache_block.rows() >= context.delta_cache_limit_rows || cache_block.bytes() >= context.delta_cache_limit_bytes; - bool is_same_schema = checkSchema(block, cache_block); + bool is_same_schema = isSameSchema(block, cache_block); if (!is_overflow && is_same_schema) { // The last cache block is available @@ -312,7 +312,7 @@ bool DeltaValueSpace::appendToCache(DMContext & context, const Block & block, si { // Create a new pack. auto last_schema = lastSchema(); - auto my_schema = (last_schema && checkSchema(block, *last_schema)) ? last_schema : std::make_shared(block.cloneEmpty()); + auto my_schema = (last_schema && isSameSchema(block, *last_schema)) ? last_schema : std::make_shared(block.cloneEmpty()); auto new_pack = DeltaPackBlock::createCachePack(my_schema); appendPackInner(new_pack); @@ -342,20 +342,19 @@ bool DeltaValueSpace::appendDeleteRange(DMContext & /*context*/, const RowKeyRan return true; } -bool DeltaValueSpace::appendRegionSnapshot(DMContext & /*context*/, - const RowKeyRange & range, - const DeltaPacks & packs, - bool clear_data_in_range) +bool DeltaValueSpace::ingestPacks(DMContext & /*context*/, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range) { std::scoped_lock lock(mutex); if (abandoned.load(std::memory_order_relaxed)) return false; + // Prepend a DeleteRange to clean data before applying packs if (clear_data_in_range) { auto p = std::make_shared(range); appendPackInner(p); } + for (auto & p : packs) { appendPackInner(p); diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index 052c86ca5e6..2580296c2dc 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -237,7 +237,7 @@ class DeltaValueSpace : public std::enable_shared_from_this, pr bool appendDeleteRange(DMContext & context, const RowKeyRange & delete_range); - bool appendRegionSnapshot(DMContext & context, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range); + bool ingestPacks(DMContext & context, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range); /// Flush the data of packs which haven't write to disk yet, and also save the metadata of packs. bool flush(DMContext & context); diff --git a/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp b/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp deleted file mode 100644 index c7b01df39e0..00000000000 --- a/dbms/src/Storages/DeltaMerge/Delta/Pack.cpp +++ /dev/null @@ -1,310 +0,0 @@ -//#include -//#include -//#include -//#include -//#include -//#include -//#include -// -//namespace DB::DM -//{ -//using PageReadFields = PageStorage::PageReadFields; -// -//// ================================================ -//// Serialize / deserialize -//// ================================================ -// -////void serializeColumn(MemoryWriteBuffer & buf, const IColumn & column, const DataTypePtr & type, size_t offset, size_t limit, bool compress) -////{ -//// CompressionMethod method = compress ? CompressionMethod::LZ4 : CompressionMethod::NONE; -//// -//// CompressedWriteBuffer compressed(buf, CompressionSettings(method)); -//// type->serializeBinaryBulkWithMultipleStreams(column, // -//// [&](const IDataType::SubstreamPath &) { return &compressed; }, -//// offset, -//// limit, -//// true, -//// {}); -//// compressed.next(); -////} -// -//void deserializeColumn(IColumn & column, const DataTypePtr & type, const ByteBuffer & data_buf, size_t rows) -//{ -// ReadBufferFromMemory buf(data_buf.begin(), data_buf.size()); -// CompressedReadBuffer compressed(buf); -// type->deserializeBinaryBulkWithMultipleStreams(column, // -// [&](const IDataType::SubstreamPath &) { return &compressed; }, -// rows, -// (double)(data_buf.size()) / rows, -// true, -// {}); -//} -// -//inline void serializePack(const Pack & pack, const BlockPtr & schema, WriteBuffer & buf) -//{ -// writeIntBinary(pack.rows, buf); -// writeIntBinary(pack.bytes, buf); -// pack.delete_range.serialize(buf); -// writeIntBinary(pack.data_page, buf); -// if (schema) -// { -// writeIntBinary((UInt32)schema->columns(), buf); -// for (auto & col : *pack.schema) -// { -// writeIntBinary(col.column_id, buf); -// writeStringBinary(col.name, buf); -// writeStringBinary(col.type->getName(), buf); -// } -// } -// else -// { -// writeIntBinary((UInt32)0, buf); -// } -//} -// -//inline PackPtr deserializePack(ReadBuffer & buf, UInt64 version) -//{ -// auto pack = std::make_shared(); -// pack->saved = true; // Must be true, otherwise it should not be here. -// pack->appendable = false; // Must be false, otherwise it should not be here. -// readIntBinary(pack->rows, buf); -// readIntBinary(pack->bytes, buf); -// if (version == 1) -// { -// HandleRange range; -// readPODBinary(range, buf); -// pack->delete_range = RowKeyRange::fromHandleRange(range); -// } -// else -// pack->delete_range = RowKeyRange::deserialize(buf); -// readIntBinary(pack->data_page, buf); -// UInt32 column_size; -// readIntBinary(column_size, buf); -// if (column_size != 0) -// { -// auto schema = std::make_shared(); -// for (size_t i = 0; i < column_size; ++i) -// { -// Int64 column_id; -// String name; -// String type_name; -// readIntBinary(column_id, buf); -// readStringBinary(name, buf); -// readStringBinary(type_name, buf); -// schema->insert(ColumnWithTypeAndName({}, DataTypeFactory::instance().get(type_name), name, column_id)); -// } -// pack->setSchema(schema); -// } -// return pack; -//} -// -//void serializeSavedPacks(WriteBuffer & buf, const Packs & packs) -//{ -// size_t saved_packs = std::find_if(packs.begin(), packs.end(), [](const PackPtr & p) { return !p->isSaved(); }) - packs.begin(); -// -// writeIntBinary(DeltaValueSpace_OLD::CURRENT_VERSION, buf); // Add binary version -// writeIntBinary(saved_packs, buf); -// BlockPtr last_schema; -// -// for (auto & pack : packs) -// { -// if (!pack->isSaved()) -// break; -// // Do not encode the schema if it is the same as previous one. -// if (pack->isDeleteRange()) -// serializePack(*pack, nullptr, buf); -// else -// { -// if (unlikely(!pack->schema)) -// throw Exception("A data pack without schema: " + pack->toString(), ErrorCodes::LOGICAL_ERROR); -// if (pack->schema != last_schema) -// { -// serializePack(*pack, pack->schema, buf); -// last_schema = pack->schema; -// } -// else -// { -// serializePack(*pack, nullptr, buf); -// } -// } -// } -//} -// -//Packs deserializePacks(ReadBuffer & buf) -//{ -// // Check binary version -// UInt64 version; -// readIntBinary(version, buf); -// if (version > DeltaValueSpace_OLD::CURRENT_VERSION) -// throw Exception("Pack binary version not match: " + DB::toString(version), ErrorCodes::LOGICAL_ERROR); -// size_t size; -// readIntBinary(size, buf); -// Packs packs; -// BlockPtr last_schema; -// for (size_t i = 0; i < (size_t)size; ++i) -// { -// auto pack = deserializePack(buf, version); -// if (!pack->isDeleteRange()) -// { -// if (!pack->schema) -// pack->setSchema(last_schema); -// else -// last_schema = pack->schema; -// } -// packs.push_back(pack); -// } -// return packs; -//} -// -//String packsToString(const Packs & packs) -//{ -// String packs_info = "["; -// for (auto & p : packs) -// { -// packs_info += (p->isDeleteRange() ? "DEL" : "INS_" + DB::toString(p->rows)) + (p->isSaved() ? "_S," : "_N,"); -// } -// if (!packs.empty()) -// packs_info.erase(packs_info.size() - 1); -// packs_info += "]"; -// return packs_info; -//} -// -//Block readPackFromCache(const PackPtr & pack) -//{ -// std::scoped_lock lock(pack->cache->mutex); -// -// auto & cache_block = pack->cache->block; -// MutableColumns columns = cache_block.cloneEmptyColumns(); -// for (size_t i = 0; i < cache_block.columns(); ++i) -// columns[i]->insertRangeFrom(*cache_block.getByPosition(i).column, 0, pack->rows); -// return cache_block.cloneWithColumns(std::move(columns)); -//} -// -//Columns readPackFromCache(const PackPtr & pack, const ColumnDefines & column_defines, size_t col_start, size_t col_end) -//{ -// if (unlikely(!(pack->cache))) -// { -// String msg = " Not a cache pack: " + pack->toString(); -// LOG_ERROR(&Logger::get(__FUNCTION__), msg); -// throw Exception(msg); -// } -// -// // TODO: should be able to use cache data directly, without copy. -// std::scoped_lock lock(pack->cache->mutex); -// -// const auto & cache_block = pack->cache->block; -// if constexpr (0) -// { -// if (pack->schema == nullptr || !checkSchema(cache_block, *pack->schema)) -// { -// const String pack_schema_str = pack->schema ? pack->schema->dumpStructure() : "(none)"; -// const String cache_schema_str = cache_block.dumpStructure(); -// throw Exception("Pack[" + pack->toString() + "] schema not match its cache_block! pack: " + pack_schema_str -// + ", cache: " + cache_schema_str, -// ErrorCodes::LOGICAL_ERROR); -// } -// } -// Columns columns; -// for (size_t i = col_start; i < col_end; ++i) -// { -// const auto & cd = column_defines[i]; -// if (auto it = pack->colid_to_offset.find(cd.id); it != pack->colid_to_offset.end()) -// { -// auto col_offset = it->second; -// // Copy data from cache -// auto [type, col_data] = pack->getDataTypeAndEmptyColumn(cd.id); -// col_data->insertRangeFrom(*cache_block.getByPosition(col_offset).column, 0, pack->rows); -// // Cast if need -// auto col_converted = convertColumnByColumnDefineIfNeed(type, std::move(col_data), cd); -// columns.push_back(std::move(col_converted)); -// } -// else -// { -// ColumnPtr column = createColumnWithDefaultValue(cd, pack->rows); -// columns.emplace_back(std::move(column)); -// } -// } -// return columns; -//} -// -//Block readPackFromDisk(const PackPtr & pack, const PageReader & page_reader) -//{ -// auto & schema = *pack->schema; -// -// PageReadFields fields; -// fields.first = pack->data_page; -// for (size_t i = 0; i < schema.columns(); ++i) -// fields.second.push_back(i); -// -// auto page_map = page_reader.read({fields}); -// auto page = page_map[pack->data_page]; -// -// auto columns = schema.cloneEmptyColumns(); -// -// if (unlikely(columns.size() != page.fieldSize())) -// throw Exception("Column size and field size not the same"); -// -// for (size_t index = 0; index < schema.columns(); ++index) -// { -// auto data_buf = page.getFieldData(index); -// auto & type = schema.getByPosition(index).type; -// auto & column = columns[index]; -// deserializeColumn(*column, type, data_buf, pack->rows); -// } -// -// return schema.cloneWithColumns(std::move(columns)); -//} -// -//Columns readPackFromDisk(const PackPtr & pack, // -// const PageReader & page_reader, -// const ColumnDefines & column_defines, -// size_t col_start, -// size_t col_end) -//{ -// const size_t num_columns_read = col_end - col_start; -// -// Columns columns(num_columns_read); // allocate empty columns -// -// PageReadFields fields; -// fields.first = pack->data_page; -// for (size_t index = col_start; index < col_end; ++index) -// { -// const auto & cd = column_defines[index]; -// if (auto it = pack->colid_to_offset.find(cd.id); it != pack->colid_to_offset.end()) -// { -// auto col_index = it->second; -// fields.second.push_back(col_index); -// } -// else -// { -// // New column after ddl is not exist in this pack, fill with default value -// columns[index - col_start] = createColumnWithDefaultValue(cd, pack->rows); -// } -// } -// -// auto page_map = page_reader.read({fields}); -// Page page = page_map[pack->data_page]; -// for (size_t index = col_start; index < col_end; ++index) -// { -// const size_t index_in_read_columns = index - col_start; -// if (columns[index_in_read_columns] != nullptr) -// { -// // the column is fill with default values. -// continue; -// } -// auto col_id = column_defines[index].id; -// auto col_index = pack->colid_to_offset[col_id]; -// auto data_buf = page.getFieldData(col_index); -// -// const auto & cd = column_defines[index]; -// // Deserialize column by pack's schema -// auto [type, col_data] = pack->getDataTypeAndEmptyColumn(cd.id); -// deserializeColumn(*col_data, type, data_buf, pack->rows); -// -// columns[index_in_read_columns] = convertColumnByColumnDefineIfNeed(type, std::move(col_data), cd); -// } -// -// return columns; -//} -// -//} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Delta/Pack.h b/dbms/src/Storages/DeltaMerge/Delta/Pack.h deleted file mode 100644 index 6484d5bbdbc..00000000000 --- a/dbms/src/Storages/DeltaMerge/Delta/Pack.h +++ /dev/null @@ -1,41 +0,0 @@ -//#pragma once -// -//#include -//#include -//#include -//#include -//#include -// -//namespace DB -//{ -// -//class MemoryWriteBuffer; -// -//namespace DM -//{ -// -//static constexpr size_t PACK_SERIALIZE_BUFFER_SIZE = 65536; -// -//void serializeColumn(MemoryWriteBuffer & buf, const IColumn & column, const DataTypePtr & type, size_t offset, size_t limit, bool compress); -// -//void serializeSavedPacks(WriteBuffer & buf, const Packs & packs); -//Packs deserializePacks(ReadBuffer & buf); -// -//// Debugging string -//String packsToString(const Packs & packs); -// -//// Read a block from cache / disk according to `pack->schema`. Only used by Compact Delta. -//Block readPackFromCache(const PackPtr & pack); -//Block readPackFromDisk(const PackPtr & pack, const PageReader & page_reader); -// -//// Read a block of columns in `column_defines` from cache / disk, -//// if `pack->schema` is not match with `column_defines`, take good care of DDL cast -//Columns readPackFromCache(const PackPtr & pack, const ColumnDefines & column_defines, size_t col_start, size_t col_end); -//Columns readPackFromDisk(const PackPtr & pack, // -// const PageReader & page_reader, -// const ColumnDefines & column_defines, -// size_t col_start, -// size_t col_end); -// -//} // namespace DM -//} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp b/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp index 8c3209a3e61..fbc6bb3857b 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/Snapshot.cpp @@ -200,7 +200,7 @@ Block DeltaValueReader::readPKVersion(size_t offset, size_t limit) Block block; for (size_t i = 0; i < 2; ++i) { - auto cd = (*col_defs)[i]; + const auto & cd = (*col_defs)[i]; block.insert(ColumnWithTypeAndName(std::move(cols[i]), cd.type, cd.name, cd.id)); } return block; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMerge.h b/dbms/src/Storages/DeltaMerge/DeltaMerge.h index 3ec56f0ac17..41fc2afcb29 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMerge.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMerge.h @@ -514,7 +514,7 @@ class DeltaMergeBlockInputStream final : public SkippableBlockInputStream, Alloc use_stable_rows = delta_index_it.getSid() - prev_sid; } } -}; // namespace DM +}; } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h b/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h index 793218376a4..d5ece791928 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeHelpers.h @@ -207,7 +207,7 @@ inline bool hasColumn(const ColumnDefines & columns, const ColId & col_id) } template -inline bool checkSchema(const Block & a, const Block & b) +inline bool isSameSchema(const Block & a, const Block & b) { if (a.columns() != b.columns()) return false; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index c1fd71f7087..68da582a8cb 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -119,7 +119,7 @@ namespace { // Actually we will always store a column of `_tidb_rowid`, no matter it // exist in `table_columns` or not. -ColumnDefinesPtr getStoreColumns(const ColumnDefines & table_columns, bool is_common_handle) +ColumnDefinesPtr generateStoreColumns(const ColumnDefines & table_columns, bool is_common_handle) { auto columns = std::make_shared(); // First three columns are always _tidb_rowid, _INTERNAL_VERSION, _INTERNAL_DELMARK @@ -176,7 +176,7 @@ DeltaMergeStore::DeltaMergeStore(Context & db_context, } original_table_header = std::make_shared(toEmptyBlock(original_table_columns)); - store_columns = getStoreColumns(original_table_columns, is_common_handle); + store_columns = generateStoreColumns(original_table_columns, is_common_handle); auto dm_context = newDMContext(db_context, db_context.getSettingsRef()); @@ -384,20 +384,9 @@ inline Block getSubBlock(const Block & block, size_t offset, size_t limit) } } -void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_settings, const Block & to_write) +// Add an extra handle column if handle reused the original column data. +Block DeltaMergeStore::addExtraColumnIfNeed(const Context & db_context, Block && block) const { - LOG_TRACE(log, __FUNCTION__ << " table: " << db_name << "." << table_name << ", rows: " << to_write.rows()); - - EventRecorder write_block_recorder(ProfileEvents::DMWriteBlock, ProfileEvents::DMWriteBlockNS); - - const auto rows = to_write.rows(); - if (rows == 0) - return; - - auto dm_context = newDMContext(db_context, db_settings); - Block block = to_write; - - // Add an extra handle column, if handle reused the original column data. if (pkIsHandle()) { auto handle_pos = getPosByColumnId(block, original_table_handle_define.id); @@ -406,8 +395,25 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ EXTRA_HANDLE_COLUMN_NAME, EXTRA_HANDLE_COLUMN_INT_TYPE, EXTRA_HANDLE_COLUMN_INT_TYPE->createColumn()); + // Fill the new column with data in column[handle_pos] FunctionToInt64::create(db_context)->execute(block, {handle_pos}, block.columns() - 1); } + return std::move(block); +} + +void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_settings, Block && to_write) +{ + LOG_TRACE(log, __FUNCTION__ << " table: " << db_name << "." << table_name << ", rows: " << to_write.rows()); + + EventRecorder write_block_recorder(ProfileEvents::DMWriteBlock, ProfileEvents::DMWriteBlockNS); + + const auto rows = to_write.rows(); + if (rows == 0) + return; + + auto dm_context = newDMContext(db_context, db_settings); + Block block = addExtraColumnIfNeed(db_context, std::move(to_write)); + const auto bytes = block.bytes(); { @@ -515,14 +521,43 @@ void DeltaMergeStore::write(const Context & db_context, const DB::Settings & db_ checkSegmentUpdate(dm_context, segment, ThreadType::Write); } -void DeltaMergeStore::writeRegionSnapshot(const DMContextPtr & dm_context, - const RowKeyRange & range, - std::vector file_ids, - bool clear_data_in_range) +std::tuple DeltaMergeStore::preAllocateIngestFile() +{ + if (shutdown_called.load(std::memory_order_relaxed)) + return {}; + + auto delegator = path_pool.getStableDiskDelegator(); + auto parent_path = delegator.choosePath(); + auto new_id = storage_pool.newDataPageId(); + return {parent_path, new_id}; +} + +void DeltaMergeStore::preIngestFile(const String & parent_path, const PageId file_id, size_t file_size) +{ + if (shutdown_called.load(std::memory_order_relaxed)) + return; + + auto delegator = path_pool.getStableDiskDelegator(); + delegator.addDTFile(file_id, file_size, parent_path); +} + +void DeltaMergeStore::ingestFiles(const DMContextPtr & dm_context, + const RowKeyRange & range, + std::vector file_ids, + bool clear_data_in_range) { + if (unlikely(shutdown_called.load(std::memory_order_relaxed))) + { + std::stringstream stream; + stream << " try to ingest files into a shutdown table: " << db_name << "." << table_name; + auto msg = stream.str(); + LOG_WARNING(log, __FUNCTION__ << msg); + throw Exception(msg); + } + LOG_INFO(log, __FUNCTION__ << " table: " << db_name << "." << table_name << ", region range:" << range.toDebugString()); - EventRecorder write_block_recorder(ProfileEvents::DMDeleteRange, ProfileEvents::DMDeleteRangeNS); + EventRecorder write_block_recorder(ProfileEvents::DMWriteFile, ProfileEvents::DMWriteFileNS); auto delegate = dm_context->path_pool.getStableDiskDelegator(); auto file_provider = dm_context->db_context.getFileProvider(); @@ -545,8 +580,8 @@ void DeltaMergeStore::writeRegionSnapshot(const DMContextPtr & dm_context, } LOG_DEBUG(log, - "[Write Region Snapshot] table: " << db_name << "." << table_name << ", rows: " << rows << ", bytes: " << bytes - << ", bytes on disk" << bytes_on_disk); + __FUNCTION__ << " table: " << db_name << "." << table_name << ", rows: " << rows << ", bytes: " << bytes << ", bytes on disk" + << bytes_on_disk); Segments updated_segments; RowKeyRange cur_range = range; @@ -622,9 +657,16 @@ void DeltaMergeStore::writeRegionSnapshot(const DMContextPtr & dm_context, // they are visible for readers who require file_ids to be found in PageStorage. wbs.writeLogAndData(); - if (segment->writeRegionSnapshot(*dm_context, range.shrink(segment_range), packs, clear_data_in_range)) + if (segment->ingestPacks(*dm_context, range.shrink(segment_range), packs, clear_data_in_range)) { updated_segments.push_back(segment); + // Enable gc for DTFile once it has been committed. + for (size_t index = 0; index < my_file_used.size(); ++index) + { + auto & file = files[index]; + if (my_file_used[index]) + file->enableGC(); + } file_used.swap(my_file_used); break; } @@ -638,6 +680,9 @@ void DeltaMergeStore::writeRegionSnapshot(const DMContextPtr & dm_context, cur_range.setEnd(range.end); } + GET_METRIC(dm_context->metrics, tiflash_storage_throughput_bytes, type_ingest).Increment(bytes); + GET_METRIC(dm_context->metrics, tiflash_storage_throughput_rows, type_ingest).Increment(rows); + flushCache(dm_context, range); for (auto & segment : updated_segments) @@ -686,7 +731,6 @@ void DeltaMergeStore::deleteRange(const Context & db_context, const DB::Settings segment_range = segment->getRowKeyRange(); // Write could fail, because other threads could already updated the instance. Like split/merge, merge delta. - if (segment->write(*dm_context, delete_range.shrink(segment_range))) { updated_segments.push_back(segment); @@ -1248,8 +1292,7 @@ bool DeltaMergeStore::handleBackgroundTask(bool heavy) segmentMerge(*task.dm_context, task.segment, task.next_segment, false); type = ThreadType::BG_Merge; break; - case MergeDelta: - { + case MergeDelta: { FAIL_POINT_PAUSE(FailPoints::pause_before_dt_background_delta_merge); left = segmentMergeDelta(*task.dm_context, task.segment, TaskRunThread::Thread_BG_Thread_Pool); type = ThreadType::BG_MergeDelta; @@ -1882,7 +1925,7 @@ void DeltaMergeStore::applyAlters(const AlterCommands & commands, } } - auto new_store_columns = getStoreColumns(new_original_table_columns, is_common_handle); + auto new_store_columns = generateStoreColumns(new_original_table_columns, is_common_handle); original_table_columns.swap(new_original_table_columns); store_columns.swap(new_store_columns); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index abd0963131b..38af1a8d115 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -285,32 +285,40 @@ class DeltaMergeStore : private boost::noncopyable // Stop all background tasks. void shutdown(); - void write(const Context & db_context, const DB::Settings & db_settings, const Block & block); - - void writeRegionSnapshot(const DMContextPtr & dm_context, // - const RowKeyRange & range, - std::vector file_ids, - bool clear_data_in_range); - - void writeRegionSnapshot(const Context & db_context, // - const DB::Settings & db_settings, - const RowKeyRange & range, - std::vector file_ids, - bool clear_data_in_range) + Block addExtraColumnIfNeed(const Context & db_context, Block && block) const; + + void write(const Context & db_context, const DB::Settings & db_settings, Block && block); + + void deleteRange(const Context & db_context, const DB::Settings & db_settings, const RowKeyRange & delete_range); + + std::tuple preAllocateIngestFile(); + + void preIngestFile(const String & parent_path, const PageId file_id, size_t file_size); + + void ingestFiles(const DMContextPtr & dm_context, // + const RowKeyRange & range, + std::vector file_ids, + bool clear_data_in_range); + + void ingestFiles(const Context & db_context, // + const DB::Settings & db_settings, + const RowKeyRange & range, + std::vector file_ids, + bool clear_data_in_range) { auto dm_context = newDMContext(db_context, db_settings); - return writeRegionSnapshot(dm_context, range, file_ids, clear_data_in_range); + return ingestFiles(dm_context, range, file_ids, clear_data_in_range); } - void deleteRange(const Context & db_context, const DB::Settings & db_settings, const RowKeyRange & delete_range); - + /// Read all rows without MVCC filtering BlockInputStreams readRaw(const Context & db_context, const DB::Settings & db_settings, const ColumnDefines & column_defines, size_t num_streams, const SegmentIdSet & read_segments = {}); - /// ranges should be sorted and merged already. + /// Read rows with MVCC filtering + /// `sorted_ranges` should be already sorted and merged BlockInputStreams read(const Context & db_context, const DB::Settings & db_settings, const ColumnDefines & columns_to_read, @@ -342,6 +350,11 @@ class DeltaMergeStore : private boost::noncopyable ColumnID & max_column_id_used, const Context & context); + const ColumnDefinesPtr getStoreColumns() const + { + std::shared_lock lock(read_write_mutex); + return store_columns; + } const ColumnDefines & getTableColumns() const { return original_table_columns; } const ColumnDefine & getHandle() const { return original_table_handle_define; } BlockPtr getHeader() const; @@ -369,9 +382,9 @@ class DeltaMergeStore : private boost::noncopyable RegionSplitRes getRegionSplitPoint(DMContext & dm_context, const RowKeyRange & check_range, size_t max_region_size, size_t split_size); +private: DMContextPtr newDMContext(const Context & db_context, const DB::Settings & db_settings); -private: bool pkIsHandle() const { return original_table_handle_define.id != EXTRA_HANDLE_COLUMN_ID; } void waitForWrite(const DMContextPtr & context, const SegmentPtr & segment); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp index 60c65f032b7..38b927daef1 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp @@ -149,7 +149,7 @@ bool DMFile::isColIndexExist(const ColId & col_id) const { if (isSingleFileMode()) { - const auto & index_identifier = DMFile::colIndexFileName(DMFile::getFileNameBase(col_id)); + const auto index_identifier = DMFile::colIndexFileName(DMFile::getFileNameBase(col_id)); return isSubFileExists(index_identifier); } else diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.h b/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.h index 6752e0a72e9..1ec2aaa7b32 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileBlockOutputStream.h @@ -8,24 +8,38 @@ namespace DB { namespace DM { + +/// The output stream for writing block to DTFile. +/// +/// Note that we will filter block by `RSOperatorPtr` while reading, so the +/// blocks output to DTFile must be bounded by primary key, or we will get +/// wrong results by filtering. +/// You can use `PKSquashingBlockInputStream` to reorganize the boundary of +/// blocks. class DMFileBlockOutputStream { public: + using Flags = DMFileWriter::Flags; + DMFileBlockOutputStream(const Context & context, const DMFilePtr & dmfile, const ColumnDefines & write_columns, - bool need_rate_limit = false) - : writer(dmfile, - write_columns, - context.getSettingsRef().min_compress_block_size, - context.getSettingsRef().max_compress_block_size, - // context.chooseCompressionSettings(0, 0), TODO: should enable this, and make unit testes work. - CompressionSettings(CompressionMethod::LZ4), - context.getFileProvider(), - need_rate_limit ? context.getRateLimiter() : nullptr) + const Flags flags = Flags()) + : writer( + dmfile, + write_columns, + context.getFileProvider(), + flags.needRateLimit() ? context.getRateLimiter() : nullptr, + DMFileWriter::Options{ + CompressionMethod::LZ4, // context.chooseCompressionSettings(0, 0), TODO: should enable this, and make unit testes work. + context.getSettingsRef().min_compress_block_size, + context.getSettingsRef().max_compress_block_size, + flags}) { } + const DMFilePtr getFile() const { return writer.getFile(); } + using BlockProperty = DMFileWriter::BlockProperty; void write(const Block & block, const BlockProperty & block_property) { writer.write(block, block_property); } diff --git a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h index 3e8996b6135..fe2b433e728 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFilePackFilter.h @@ -163,15 +163,11 @@ class DMFilePackFilter Float64 filter_rate = (Float64)(after_read_packs - after_filter) * 100 / after_read_packs; LOG_DEBUG(log, - "RSFilter exclude rate is nan, after_pk: " << after_pk << ", after_read_packs: " << after_read_packs << ", after_filter: " - << after_filter << ", handle_range: " << rowkey_range.toDebugString() - << ", read_packs: " << ((!read_packs) ? 0 : read_packs->size()) - << ", pack_count: " << pack_count); - - if (isnan(filter_rate)) - LOG_DEBUG(log, "RSFilter exclude rate: nan"); - else - LOG_DEBUG(log, "RSFilter exclude rate: " << DB::toString(filter_rate, 2)); + "RSFilter exclude rate: " << ((after_read_packs == 0) ? "nan" : DB::toString(filter_rate, 2)) + << ", after_pk: " << after_pk << ", after_read_packs: " << after_read_packs + << ", after_filter: " << after_filter << ", handle_range: " << rowkey_range.toDebugString() + << ", read_packs: " << ((!read_packs) ? 0 : read_packs->size()) + << ", pack_count: " << pack_count); } friend class DMFileReader; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp index 586ffae9f05..1ad99630840 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp @@ -8,33 +8,30 @@ namespace DB namespace DM { -DMFileWriter::DMFileWriter(const DMFilePtr & dmfile_, - const ColumnDefines & write_columns_, - size_t min_compress_block_size_, - size_t max_compress_block_size_, - const CompressionSettings & compression_settings_, - const FileProviderPtr & file_provider_, - const RateLimiterPtr & rate_limiter_) +DMFileWriter::DMFileWriter(const DMFilePtr & dmfile_, + const ColumnDefines & write_columns_, + const FileProviderPtr & file_provider_, + const RateLimiterPtr & rate_limiter_, + const DMFileWriter::Options & options_) : dmfile(dmfile_), write_columns(write_columns_), - min_compress_block_size(min_compress_block_size_), - max_compress_block_size(max_compress_block_size_), - compression_settings(compression_settings_), - single_file_mode(dmfile->isSingleFileMode()), + options(options_, dmfile), // assume pack_stat_file is the first file created inside DMFile // it will create encryption info for the whole DMFile - pack_stat_file(single_file_mode ? nullptr - : createWriteBufferFromFileBaseByFileProvider(file_provider_, - dmfile->packStatPath(), - dmfile->encryptionPackStatPath(), - true, - rate_limiter_, - 0, - 0, - max_compress_block_size)), - single_file_stream(!single_file_mode ? nullptr - : new SingleFileStream( - dmfile_, compression_settings_, max_compress_block_size_, file_provider_, rate_limiter_)), + pack_stat_file((options.flags.isSingleFile()) // + ? nullptr + : createWriteBufferFromFileBaseByFileProvider(file_provider_, + dmfile->packStatPath(), + dmfile->encryptionPackStatPath(), + true, + rate_limiter_, + 0, + 0, + options.max_compress_block_size)), + single_file_stream((!options.flags.isSingleFile()) + ? nullptr + : new SingleFileStream( + dmfile_, options.compression_settings, options.max_compress_block_size, file_provider_, rate_limiter_)), file_provider(file_provider_), rate_limiter(rate_limiter_) { @@ -46,7 +43,7 @@ DMFileWriter::DMFileWriter(const DMFilePtr & dmfile_, /// for handle column always generate index bool do_index = cd.id == EXTRA_HANDLE_COLUMN_ID || cd.type->isInteger() || cd.type->isDateOrDateTime(); - if (single_file_mode) + if (options.flags.isSingleFile()) { if (do_index) { @@ -76,8 +73,8 @@ void DMFileWriter::addStreams(ColId col_id, DataTypePtr type, bool do_index) auto stream = std::make_unique(dmfile, // stream_name, type, - compression_settings, - max_compress_block_size, + options.compression_settings, + options.max_compress_block_size, file_provider, rate_limiter, IDataType::isNullMap(substream_path) ? false : do_index); @@ -110,7 +107,7 @@ void DMFileWriter::write(const Block & block, const BlockProperty & block_proper stat.first_tag = (UInt8)(col->get64(0)); } - if (!single_file_mode) + if (!options.flags.isSingleFile()) { writePODBinary(stat, *pack_stat_file); } @@ -125,7 +122,7 @@ void DMFileWriter::write(const Block & block, const BlockProperty & block_proper void DMFileWriter::finalize() { - if (!single_file_mode) + if (!options.flags.isSingleFile()) { pack_stat_file->sync(); } @@ -135,7 +132,7 @@ void DMFileWriter::finalize() finalizeColumn(cd.id, cd.type); } - if (single_file_mode) + if (options.flags.isSingleFile()) { dmfile->finalizeForSingleFileMode(single_file_stream->plain_hashing); single_file_stream->flush(); @@ -150,7 +147,7 @@ void DMFileWriter::writeColumn(ColId col_id, const IDataType & type, const IColu { size_t rows = column.size(); - if (single_file_mode) + if (options.flags.isSingleFile()) { auto callback = [&](const IDataType::SubstreamPath & substream) { size_t offset_in_compressed_file = single_file_stream->plain_hashing.count(); @@ -223,7 +220,7 @@ void DMFileWriter::writeColumn(ColId col_id, const IDataType & type, const IColu stream->minmaxes->addPack(column, del_mark); /// There could already be enough data to compress into the new block. - if (stream->original_hashing.offset() >= min_compress_block_size) + if (stream->original_hashing.offset() >= options.min_compress_block_size) stream->original_hashing.next(); auto offset_in_compressed_block = stream->original_hashing.offset(); @@ -261,7 +258,7 @@ void DMFileWriter::finalizeColumn(ColId col_id, DataTypePtr type) { size_t bytes_written = 0; - if (single_file_mode) + if (options.flags.isSingleFile()) { auto callback = [&](const IDataType::SubstreamPath & substream) { const auto stream_name = DMFile::getFileNameBase(col_id, substream); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h index 8b09a8f1d0a..4f4a0332857 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h @@ -136,18 +136,63 @@ class DMFileWriter size_t gc_hint_version; }; + struct Flags + { + private: + static constexpr size_t IS_SINGLE_FILE = 0x01; + static constexpr size_t NEED_RATE_LIMIT = 0x02; + + size_t value; + + public: + Flags() : value(0x0) {} + + inline void setSingleFile(bool v) { value = (v ? (value | IS_SINGLE_FILE) : (value & ~IS_SINGLE_FILE)); } + inline bool isSingleFile() const { return (value & IS_SINGLE_FILE); } + inline void setRateLimit(bool v) { value = (v ? (value | NEED_RATE_LIMIT) : (value & ~NEED_RATE_LIMIT)); } + inline bool needRateLimit() const { return (value & NEED_RATE_LIMIT); } + }; + + struct Options + { + CompressionSettings compression_settings; + size_t min_compress_block_size; + size_t max_compress_block_size; + Flags flags; + + Options() = default; + + Options(CompressionSettings compression_settings_, size_t min_compress_block_size_, size_t max_compress_block_size_, Flags flags_) + : compression_settings(compression_settings_), + min_compress_block_size(min_compress_block_size_), + max_compress_block_size(max_compress_block_size_), + flags(flags_) + { + } + + Options(const Options & from, const DMFilePtr & file) + : compression_settings(from.compression_settings), + min_compress_block_size(from.min_compress_block_size), + max_compress_block_size(from.max_compress_block_size), + flags(from.flags) + { + flags.setSingleFile(file->isSingleFileMode()); + } + }; + + public: - DMFileWriter(const DMFilePtr & dmfile_, - const ColumnDefines & write_columns_, - size_t min_compress_block_size_, - size_t max_compress_block_size_, - const CompressionSettings & compression_settings_, - const FileProviderPtr & file_provider_, - const RateLimiterPtr & rate_limiter_); + DMFileWriter(const DMFilePtr & dmfile_, + const ColumnDefines & write_columns_, + const FileProviderPtr & file_provider_, + const RateLimiterPtr & rate_limiter_, + const Options & options_); void write(const Block & block, const BlockProperty & block_property); void finalize(); + const DMFilePtr getFile() const { return dmfile; } + private: void finalizeColumn(ColId col_id, DataTypePtr type); void writeColumn(ColId col_id, const IDataType & type, const IColumn & column, const ColumnVector * del_mark); @@ -158,12 +203,9 @@ class DMFileWriter void addStreams(ColId col_id, DataTypePtr type, bool do_index); private: - DMFilePtr dmfile; - ColumnDefines write_columns; - size_t min_compress_block_size; - size_t max_compress_block_size; - CompressionSettings compression_settings; - const bool single_file_mode; + DMFilePtr dmfile; + ColumnDefines write_columns; + Options options; ColumnStreams column_streams; diff --git a/dbms/src/Storages/DeltaMerge/ReorganizeBlockInputStream.h b/dbms/src/Storages/DeltaMerge/PKSquashingBlockInputStream.h similarity index 60% rename from dbms/src/Storages/DeltaMerge/ReorganizeBlockInputStream.h rename to dbms/src/Storages/DeltaMerge/PKSquashingBlockInputStream.h index 3a1962beed6..31f698d18a7 100644 --- a/dbms/src/Storages/DeltaMerge/ReorganizeBlockInputStream.h +++ b/dbms/src/Storages/DeltaMerge/PKSquashingBlockInputStream.h @@ -1,38 +1,53 @@ #pragma once #include +#include #include +#include namespace DB { namespace DM { -/// Reorganize the boundary of blocks. -/// Note that child must be a sorted input stream with increasing pk column. -class ReorganizeBlockInputStream final : public IBlockInputStream +/// Reorganize the boundary of blocks. The rows with the same primary key(s) will be squashed +/// into the same output block. The output blocks are sorted by increasing pk && version. +/// Note that the `child` must be a sorted input stream with increasing pk && version. If you are +/// not sure the child stream is sorted with increasing pk && version, set `need_extra_sort` to +/// be `true`. +template +class PKSquashingBlockInputStream final : public IBlockInputStream { public: - ReorganizeBlockInputStream(BlockInputStreamPtr child, String pk_column_name_) - : sorted_input_stream(std::move(child)), pk_column_name(std::move(pk_column_name_)) + PKSquashingBlockInputStream(BlockInputStreamPtr child, ColId pk_column_id_, bool is_common_handle_) + : sorted_input_stream(child), + pk_column_id(pk_column_id_), + is_common_handle(is_common_handle_) { assert(sorted_input_stream != nullptr); cur_block = {}; - if constexpr (DM_RUN_CHECK) - { - // Sanity check for existence of pk column - Block header = sorted_input_stream->getHeader(); - if (!header.has(pk_column_name)) - { - throw Exception("Try to write block to Pack without pk column", ErrorCodes::LOGICAL_ERROR); - } - is_common_handle = !header.getByName(pk_column_name).type->isInteger(); - } + children.push_back(child); } - String getName() const override { return "ReorganizeBlockBoundary"; } + String getName() const override { return "PKSquashing"; } Block getHeader() const override { return sorted_input_stream->getHeader(); } + void readPrefix() override + { + forEachChild([](IBlockInputStream & child) { + child.readPrefix(); + return false; + }); + } + + void readSuffix() override + { + forEachChild([](IBlockInputStream & child) { + child.readSuffix(); + return false; + }); + } + Block read() override { if (first_read) @@ -43,16 +58,24 @@ class ReorganizeBlockInputStream final : public IBlockInputStream cur_block = next_block; if (!cur_block) - return cur_block; + return finializeBlock(std::move(cur_block)); while (true) { next_block = DB::DM::readNextBlock(sorted_input_stream); - const size_t cut_offset = findCutOffsetInNextBlock(cur_block, next_block, pk_column_name, is_common_handle); +#ifndef NDEBUG + if (next_block && !isSameSchema(cur_block, next_block)) + { + throw Exception("schema not match! [cur_block=" + cur_block.dumpStructure() + "] [next_block=" + next_block.dumpStructure() + + "]"); + } +#endif + + const size_t cut_offset = findCutOffsetInNextBlock(cur_block, next_block, pk_column_id, is_common_handle); if (unlikely(cut_offset == 0)) // There is no pk overlap between `cur_block` and `next_block`, or `next_block` is empty, just return `cur_block`. - return cur_block; + return finializeBlock(std::move(cur_block)); else { const size_t next_block_nrows = next_block.rows(); @@ -77,7 +100,7 @@ class ReorganizeBlockInputStream final : public IBlockInputStream if (cut_offset != next_block_nrows) { // We merge some rows to `cur_block`, return it. - return cur_block; + return finializeBlock(std::move(cur_block)); } // else we merge all rows from `next_block` to `cur_block`, continue to check if we should merge more blocks. } @@ -86,16 +109,16 @@ class ReorganizeBlockInputStream final : public IBlockInputStream private: static size_t - findCutOffsetInNextBlock(const Block & cur_block, const Block & next_block, const String & pk_column_name, bool is_common_handle) + findCutOffsetInNextBlock(const Block & cur_block, const Block & next_block, const ColId pk_column_id, bool is_common_handle) { assert(cur_block); if (!next_block) return 0; - auto cur_col = cur_block.getByName(pk_column_name).column; + auto cur_col = getByColumnId(cur_block, pk_column_id).column; RowKeyColumnContainer cur_rowkey_column(cur_col, is_common_handle); const auto last_curr_pk = cur_rowkey_column.getRowKeyValue(cur_col->size() - 1); - auto next_col = next_block.getByName(pk_column_name).column; + auto next_col = getByColumnId(next_block, pk_column_id).column; RowKeyColumnContainer next_rowkey_column(next_col, is_common_handle); size_t cut_offset = 0; for (/* */; cut_offset < next_col->size(); ++cut_offset) @@ -116,15 +139,28 @@ class ReorganizeBlockInputStream final : public IBlockInputStream return cut_offset; } + static Block finializeBlock(Block && block) + { + if constexpr (need_extra_sort) + { + // Sort by handle & version in ascending order. + static SortDescription sort{SortColumnDescription{EXTRA_HANDLE_COLUMN_NAME, 1, 0}, + SortColumnDescription{VERSION_COLUMN_NAME, 1, 0}}; + if (block.rows() > 1 && !isAlreadySorted(block, sort)) + stableSortBlock(block, sort); + } + return block; + } + private: BlockInputStreamPtr sorted_input_stream; - const String pk_column_name; - bool is_common_handle; + const ColId pk_column_id; Block cur_block; Block next_block; - bool first_read = true; + bool first_read = true; + const bool is_common_handle; }; } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.cpp b/dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.cpp new file mode 100644 index 00000000000..f3152312326 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.cpp @@ -0,0 +1,249 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLFORMAT_RAFT_ROW; +} // namespace ErrorCodes + +Block GenRegionBlockDatawithSchema( // + const RegionPtr &, + const std::shared_ptr &, + const DM::ColumnDefinesPtr &, + Timestamp, + bool, + TMTContext &); + +namespace DM +{ + +SSTFilesToBlockInputStream::SSTFilesToBlockInputStream( // + RegionPtr region_, + const SSTViewVec & snaps_, + const TiFlashRaftProxyHelper * proxy_helper_, + SSTFilesToBlockInputStream::StorageDeltaMergePtr ingest_storage_, + DM::ColumnDefinesPtr schema_snap_, + Timestamp gc_safepoint_, + bool force_decode_, + TMTContext & tmt_, + size_t expected_size_) + : region(std::move(region_)), + snaps(snaps_), + proxy_helper(proxy_helper_), + ingest_storage(std::move(ingest_storage_)), + schema_snap(std::move(schema_snap_)), + tmt(tmt_), + gc_safepoint(gc_safepoint_), + expected_size(expected_size_), + log(&Poco::Logger::get("SSTFilesToBlockInputStream")), + force_decode(force_decode_) +{ +} + +SSTFilesToBlockInputStream::~SSTFilesToBlockInputStream() = default; + +void SSTFilesToBlockInputStream::readPrefix() +{ + for (UInt64 i = 0; i < snaps.len; ++i) + { + auto & snapshot = snaps.views[i]; + switch (snapshot.type) + { + case ColumnFamilyType::Default: + default_cf_reader = std::make_unique(proxy_helper, snapshot); + break; + case ColumnFamilyType::Write: + write_cf_reader = std::make_unique(proxy_helper, snapshot); + break; + case ColumnFamilyType::Lock: + lock_cf_reader = std::make_unique(proxy_helper, snapshot); + break; + } + } + + process_keys.default_cf = 0; + process_keys.write_cf = 0; + process_keys.lock_cf = 0; +} + +void SSTFilesToBlockInputStream::readSuffix() +{ + // There must be no data left when we write suffix + assert(!write_cf_reader || !write_cf_reader->remained()); + assert(!default_cf_reader || !default_cf_reader->remained()); + assert(!lock_cf_reader || !lock_cf_reader->remained()); + + // reset all SSTReaders and return without writting blocks any more. + write_cf_reader.reset(); + default_cf_reader.reset(); + lock_cf_reader.reset(); +} + +Block SSTFilesToBlockInputStream::read() +{ + while (write_cf_reader && write_cf_reader->remained()) + { + // Read a key-value from write CF and continue to read key-value until + // the key that we read from write CF. + // All SST files store key-value in a sorted way so that we are able to + // scan committed rows in `region`. + auto key = write_cf_reader->key(); + auto value = write_cf_reader->value(); + region->insert(ColumnFamilyType::Write, TiKVKey(key.data, key.len), TiKVValue(value.data, value.len)); + ++process_keys.write_cf; + write_cf_reader->next(); + + if (process_keys.write_cf % expected_size == 0) + { + auto key_view = std::string_view{key.data, key.len}; + // Batch the scan from other CFs until we need to decode data + scanCF(ColumnFamilyType::Default, /*until*/ key_view); + scanCF(ColumnFamilyType::Lock, /*until*/ key_view); + + auto block = readCommitedBlock(); + if (block.rows() != 0) + return block; + // else continue to decode key-value from write CF. + } + } + // Scan all key-value pairs from other CFs + scanCF(ColumnFamilyType::Default); + scanCF(ColumnFamilyType::Lock); + + // All uncommitted data are saved in `region`, decode the last committed rows. + return readCommitedBlock(); +} + +void SSTFilesToBlockInputStream::scanCF(ColumnFamilyType cf, const std::string_view until) +{ + SSTReader * reader; + if (cf == ColumnFamilyType::Default) + reader = default_cf_reader.get(); + else if (cf == ColumnFamilyType::Lock) + reader = lock_cf_reader.get(); + else + throw Exception("Should not happen!"); + + size_t num_process_keys = 0; + while (reader && reader->remained()) + { + auto key = reader->key(); + if (until.data() == nullptr || memcmp(until.data(), key.data, std::min(until.size(), key.len)) >= 0) + { + auto value = reader->value(); + region->insert(cf, TiKVKey(key.data, key.len), TiKVValue(value.data, value.len)); + ++num_process_keys; + reader->next(); + } + else + break; + } + + if (cf == ColumnFamilyType::Default) + process_keys.default_cf += num_process_keys; + else if (cf == ColumnFamilyType::Lock) + process_keys.lock_cf += num_process_keys; +} + +Block SSTFilesToBlockInputStream::readCommitedBlock() +{ + if (is_decode_cancelled) + return {}; + + try + { + // Read block from `region`. If the schema has been updated, it will + // throw an exception with code `ErrorCodes::REGION_DATA_SCHEMA_UPDATED` + return GenRegionBlockDatawithSchema(region, ingest_storage, schema_snap, gc_safepoint, force_decode, tmt); + } + catch (DB::Exception & e) + { + if (e.code() == ErrorCodes::ILLFORMAT_RAFT_ROW) + { + // br or lighting may write illegal data into tikv, stop decoding. + LOG_WARNING(log, + "Got error while reading region committed cache: " + << e.displayText() << ". Stop decoding rows into DTFiles and keep uncommitted data in region."); + // Cancel the decoding process. + // Note that we still need to scan data from CFs and keep them in `region` + is_decode_cancelled = true; + return {}; + } + else + throw; + } +} + +/// Methods for BoundedSSTFilesToBlockInputStream + +BoundedSSTFilesToBlockInputStream::BoundedSSTFilesToBlockInputStream( // + SSTFilesToBlockInputStreamPtr child, + const ColId pk_column_id_, + const bool is_common_handle_) + : pk_column_id(pk_column_id_), is_common_handle(is_common_handle_), _raw_child(std::move(child)) +{ + // Initlize `mvcc_compact_stream` + // First refine the boundary of blocks. Note that the rows decoded from SSTFiles are sorted by primary key asc, timestamp desc + // (https://github.com/tikv/tikv/blob/v5.0.1/components/txn_types/src/types.rs#L103-L108). + // While DMVersionFilter require rows sorted by primary key asc, timestamp asc, so we need an extra sort in PKSquashing. + auto stream = std::make_shared>(_raw_child, pk_column_id, is_common_handle); + mvcc_compact_stream = std::make_unique>( + stream, *(_raw_child->schema_snap), _raw_child->gc_safepoint, is_common_handle); +} + +void BoundedSSTFilesToBlockInputStream::readPrefix() +{ + mvcc_compact_stream->readPrefix(); +} + +void BoundedSSTFilesToBlockInputStream::readSuffix() +{ + mvcc_compact_stream->readSuffix(); +} + +Block BoundedSSTFilesToBlockInputStream::read() +{ + return mvcc_compact_stream->read(); +} + +std::tuple, DM::ColumnDefinesPtr> BoundedSSTFilesToBlockInputStream::ingestingInfo() const +{ + return std::make_tuple(_raw_child->ingest_storage, _raw_child->schema_snap); +} + +SSTFilesToBlockInputStream::ProcessKeys BoundedSSTFilesToBlockInputStream::getProcessKeys() const +{ + return _raw_child->process_keys; +} + +const RegionPtr BoundedSSTFilesToBlockInputStream::getRegion() const +{ + return _raw_child->region; +} + +std::tuple // +BoundedSSTFilesToBlockInputStream::getMvccStatistics() const +{ + return std::make_tuple( + mvcc_compact_stream->getEffectiveNumRows(), mvcc_compact_stream->getNotCleanRows(), mvcc_compact_stream->getGCHintVersion()); +} + +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.h b/dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.h new file mode 100644 index 00000000000..97585d3091c --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/SSTFilesToBlockInputStream.h @@ -0,0 +1,137 @@ +#pragma once + +#include +#include +#include + +#include +#include + +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +class TMTContext; +class Region; +using RegionPtr = std::shared_ptr; + +struct SSTViewVec; +struct TiFlashRaftProxyHelper; +struct SSTReader; +class StorageDeltaMerge; + +namespace DM +{ + +struct ColumnDefine; +using ColumnDefines = std::vector; +using ColumnDefinesPtr = std::shared_ptr; + +// forward declaration +class SSTFilesToBlockInputStream; +using SSTFilesToBlockInputStreamPtr = std::shared_ptr; +class BoundedSSTFilesToBlockInputStream; +using BoundedSSTFilesToBlockInputStreamPtr = std::shared_ptr; + +class SSTFilesToBlockInputStream final : public IBlockInputStream +{ +public: + using StorageDeltaMergePtr = std::shared_ptr; + SSTFilesToBlockInputStream(RegionPtr region_, + const SSTViewVec & snaps_, + const TiFlashRaftProxyHelper * proxy_helper_, + StorageDeltaMergePtr ingest_storage_, + DM::ColumnDefinesPtr schema_snap_, + Timestamp gc_safepoint_, + bool force_decode_, + TMTContext & tmt_, + size_t expected_size_ = DEFAULT_MERGE_BLOCK_SIZE); + ~SSTFilesToBlockInputStream(); + + String getName() const override { return "SSTFilesToBlockInputStream"; } + + Block getHeader() const override { return toEmptyBlock(*schema_snap); } + + void readPrefix() override; + void readSuffix() override; + Block read() override; + +public: + struct ProcessKeys + { + size_t default_cf; + size_t write_cf; + size_t lock_cf; + + inline size_t total() const { return default_cf + write_cf + lock_cf; } + }; + +private: + void scanCF(ColumnFamilyType cf, const std::string_view until = std::string_view{}); + + Block readCommitedBlock(); + +private: + RegionPtr region; + const SSTViewVec & snaps; + const TiFlashRaftProxyHelper * proxy_helper{nullptr}; + const StorageDeltaMergePtr ingest_storage; + const DM::ColumnDefinesPtr schema_snap; + TMTContext & tmt; + const Timestamp gc_safepoint; + size_t expected_size; + Poco::Logger * log; + + using SSTReaderPtr = std::unique_ptr; + SSTReaderPtr write_cf_reader; + SSTReaderPtr default_cf_reader; + SSTReaderPtr lock_cf_reader; + + friend class BoundedSSTFilesToBlockInputStream; + + const bool force_decode; + bool is_decode_cancelled = false; + + ProcessKeys process_keys; +}; + +// Bound the blocks read from SSTFilesToBlockInputStream by column `_tidb_rowid` and +// do some calculation for the `DMFileWriter::BlockProperty` of read blocks. +class BoundedSSTFilesToBlockInputStream final +{ +public: + BoundedSSTFilesToBlockInputStream(SSTFilesToBlockInputStreamPtr child, const ColId pk_column_id_, const bool is_common_handle_); + + String getName() const { return "BoundedSSTFilesToBlockInputStream"; } + + void readPrefix(); + + void readSuffix(); + + Block read(); + + std::tuple, DM::ColumnDefinesPtr> ingestingInfo() const; + + SSTFilesToBlockInputStream::ProcessKeys getProcessKeys() const; + + const RegionPtr getRegion() const; + + // Return values: (effective rows, not clean rows, gc hint version) + std::tuple getMvccStatistics() const; + +private: + const ColId pk_column_id; + const bool is_common_handle; + + // Note that we only keep _raw_child for getting ingest info / process key, etc. All block should be + // read from `mvcc_compact_stream` + const SSTFilesToBlockInputStreamPtr _raw_child; + std::unique_ptr> mvcc_compact_stream; +}; + +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/SSTFilesToDTFilesOutputStream.cpp b/dbms/src/Storages/DeltaMerge/SSTFilesToDTFilesOutputStream.cpp new file mode 100644 index 00000000000..c2fa9c3099c --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/SSTFilesToDTFilesOutputStream.cpp @@ -0,0 +1,220 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ +extern const Event DMWriteBytes; +} + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ILLFORMAT_RAFT_ROW; +} // namespace ErrorCodes + +namespace DM +{ + +SSTFilesToDTFilesOutputStream::SSTFilesToDTFilesOutputStream( // + BoundedSSTFilesToBlockInputStreamPtr child_, + TiDB::SnapshotApplyMethod method_, + FileConvertJobType job_type_, + TMTContext & tmt_) + : child(child_), // + method(method_), + job_type(job_type_), + tmt(tmt_), + log(&Poco::Logger::get("SSTFilesToDTFilesOutputStream")) +{ +} + +SSTFilesToDTFilesOutputStream::~SSTFilesToDTFilesOutputStream() = default; + +void SSTFilesToDTFilesOutputStream::writePrefix() +{ + child->readPrefix(); + + commit_rows = 0; + watch.start(); +} + +void SSTFilesToDTFilesOutputStream::writeSuffix() +{ + child->readSuffix(); + + if (dt_stream != nullptr) + { + dt_stream->writeSuffix(); + auto dt_file = dt_stream->getFile(); + assert(!dt_file->canGC()); // The DTFile should not be able to gc until it is ingested. + // Add the DTFile to StoragePathPool so that we can restore it later + auto [ingest_storage, _schema_snap] = child->ingestingInfo(); + std::ignore = _schema_snap; + const auto bytes_written = dt_file->getBytesOnDisk(); + ingest_storage->getStore()->preIngestFile(dt_file->parentPath(), dt_file->fileId(), bytes_written); + + // Report DMWriteBytes for calculating write amplification + ProfileEvents::increment(ProfileEvents::DMWriteBytes, bytes_written); + + dt_stream.reset(); + } + + auto & ctx = tmt.getContext(); + auto metrics = ctx.getTiFlashMetrics(); + const auto process_keys = child->getProcessKeys(); + if (job_type == FileConvertJobType::ApplySnapshot) + { + GET_METRIC(metrics, tiflash_raft_command_duration_seconds, type_apply_snapshot_predecode).Observe(watch.elapsedSeconds()); + // Note that number of keys in different cf will be aggregated into one metrics + GET_METRIC(metrics, tiflash_raft_process_keys, type_apply_snapshot).Increment(process_keys.total()); + } + else + { + // Note that number of keys in different cf will be aggregated into one metrics + GET_METRIC(metrics, tiflash_raft_process_keys, type_ingest_sst).Increment(process_keys.total()); + } + LOG_INFO(log, + "Pre-handle snapshot " << child->getRegion()->toString(true) << " to " << ingest_files.size() << " DTFiles, cost " + << watch.elapsedMilliseconds() << "ms [rows=" << commit_rows + << "] [write_cf_keys=" << process_keys.write_cf << "] [default_cf_keys=" << process_keys.default_cf + << "] [lock_cf_keys=" << process_keys.lock_cf << "]"); +} + +bool SSTFilesToDTFilesOutputStream::newDTFileStream() +{ + // Generate a DMFilePtr and its DMFileBlockOutputStream + DMFileBlockOutputStream::Flags flags; + switch (method) + { + case TiDB::SnapshotApplyMethod::DTFile_Directory: + flags.setSingleFile(false); + break; + case TiDB::SnapshotApplyMethod::DTFile_Single: + flags.setSingleFile(true); + break; + default: + break; + } + + // The parent_path and file_id are generated by the storage. + auto [ingest_storage, schema_snap] = child->ingestingInfo(); + auto [parent_path, file_id] = ingest_storage->getStore()->preAllocateIngestFile(); + if (parent_path.empty()) + { + // Can no allocate path and id for storing DTFiles (the storage may be dropped / shutdown) + return false; + } + auto dt_file = DMFile::create(file_id, parent_path, flags.isSingleFile()); + LOG_INFO(log, "Create file for snapshot data [file=" << dt_file->path() << "] [single_file_mode=" << flags.isSingleFile() << "]"); + dt_stream = std::make_unique(tmt.getContext(), dt_file, *schema_snap, flags); + dt_stream->writePrefix(); + ingest_files.emplace_back(dt_file); + return true; +} + +void SSTFilesToDTFilesOutputStream::write() +{ + size_t last_effective_num_rows = 0; + size_t last_not_clean_rows = 0; + size_t cur_effective_num_rows = 0; + size_t cur_not_clean_rows = 0; + while (true) + { + + Block block = child->read(); + if (!block) + break; + if (unlikely(block.rows() == 0)) + continue; + + if (dt_stream == nullptr) + { + // If can not create DTFile stream (the storage may be dropped / shutdown), + // break the writing loop. + if (bool ok = newDTFileStream(); !ok) + { + break; + } + } + + { + // Check whether rows are sorted by handle & version in ascending order. + SortDescription sort; + sort.emplace_back(MutableSupport::tidb_pk_column_name, 1, 0); + sort.emplace_back(MutableSupport::version_column_name, 1, 0); + if (unlikely(block.rows() > 1 && !isAlreadySorted(block, sort))) + { + const String error_msg + = "The block decoded from SSTFile is not sorted by primary key and version " + child->getRegion()->toString(true); + LOG_ERROR(log, error_msg); + FieldVisitorToString visitor; + const size_t nrows = block.rows(); + for (size_t i = 0; i < nrows; ++i) + { + const auto & pk_col = block.getByName(MutableSupport::tidb_pk_column_name); + const auto & ver_col = block.getByName(MutableSupport::version_column_name); + LOG_ERROR(log, + "[Row=" << i << "/" << nrows << "] [pk=" << applyVisitor(visitor, (*pk_col.column)[i]) + << "] [ver=" << applyVisitor(visitor, (*ver_col.column)[i]) << "]"); + } + throw Exception(error_msg); + } + } + + // Write block to the output stream + DMFileBlockOutputStream::BlockProperty property; + std::tie(cur_effective_num_rows, cur_not_clean_rows, property.gc_hint_version) // + = child->getMvccStatistics(); + property.effective_num_rows = cur_effective_num_rows - last_effective_num_rows; + property.not_clean_rows = cur_not_clean_rows - last_not_clean_rows; + dt_stream->write(block, property); + + commit_rows += block.rows(); + last_effective_num_rows = cur_effective_num_rows; + last_not_clean_rows = cur_not_clean_rows; + } +} + +PageIds SSTFilesToDTFilesOutputStream::ingestIds() const +{ + PageIds ids; + for (const auto & file : ingest_files) + { + ids.emplace_back(file->fileId()); + } + return ids; +} + +void SSTFilesToDTFilesOutputStream::cancel() +{ + // Try a lightweight cleanup the file generated by this stream (marking them able to be GC-ed). + for (auto & file : ingest_files) + { + try + { + file->enableGC(); + } + catch (...) + { + tryLogCurrentException(log, "ignore exception while canceling SST files to DeltaTree files stream [file=" + file->path() + "]"); + } + } +} + +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/SSTFilesToDTFilesOutputStream.h b/dbms/src/Storages/DeltaMerge/SSTFilesToDTFilesOutputStream.h new file mode 100644 index 00000000000..be80372cfa0 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/SSTFilesToDTFilesOutputStream.h @@ -0,0 +1,91 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include + +namespace Poco +{ +class Logger; +} + +namespace DB +{ + +class TMTContext; +class Region; +using RegionPtr = std::shared_ptr; + +struct SSTViewVec; +struct TiFlashRaftProxyHelper; +struct SSTReader; +class StorageDeltaMerge; + +namespace DM +{ + +struct ColumnDefine; +using ColumnDefines = std::vector; +using ColumnDefinesPtr = std::shared_ptr; + +class DMFile; +using DMFilePtr = std::shared_ptr; +class DMFileBlockOutputStream; + +enum class FileConvertJobType +{ + ApplySnapshot, + IngestSST, +}; + + +// This class is tightly coupling with BoundedSSTFilesToBlockInputStream +// to get some info of the decoding process. +class SSTFilesToDTFilesOutputStream : private boost::noncopyable +{ +public: + using StorageDeltaMergePtr = std::shared_ptr; + SSTFilesToDTFilesOutputStream(BoundedSSTFilesToBlockInputStreamPtr child_, + TiDB::SnapshotApplyMethod method_, + FileConvertJobType job_type_, + TMTContext & tmt_); + ~SSTFilesToDTFilesOutputStream(); + + void writePrefix(); + void writeSuffix(); + void write(); + + PageIds ingestIds() const; + + // Try to cleanup the files in `ingest_files` quickly. + void cancel(); + +private: + + bool newDTFileStream(); + + // Stop the process for decoding committed data into DTFiles + void stop(); + +private: + BoundedSSTFilesToBlockInputStreamPtr child; + const TiDB::SnapshotApplyMethod method; + const FileConvertJobType job_type; + TMTContext & tmt; + Poco::Logger * log; + + std::unique_ptr dt_stream; + + std::vector ingest_files; + + size_t schema_sync_trigger_count = 0; + size_t commit_rows = 0; + Stopwatch watch; +}; + +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 044139ca5b1..b488aa3b8a7 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include @@ -74,15 +74,15 @@ namespace DM const static size_t SEGMENT_BUFFER_SIZE = 128; // More than enough. -DMFilePtr writeIntoNewDMFile(DMContext & dm_context, // - const ColumnDefinesPtr & schema_snap, - const BlockInputStreamPtr & input_stream, - UInt64 file_id, - const String & parent_path, - bool need_rate_limit) +DMFilePtr writeIntoNewDMFile(DMContext & dm_context, // + const ColumnDefinesPtr & schema_snap, + const BlockInputStreamPtr & input_stream, + UInt64 file_id, + const String & parent_path, + DMFileBlockOutputStream::Flags flags) { - auto dmfile = DMFile::create(file_id, parent_path, dm_context.db_context.getSettingsRef().dt_enable_single_file_mode_dmfile); - auto output_stream = std::make_shared(dm_context.db_context, dmfile, *schema_snap, need_rate_limit); + auto dmfile = DMFile::create(file_id, parent_path, flags.isSingleFile()); + auto output_stream = std::make_shared(dm_context.db_context, dmfile, *schema_snap, flags); auto * mvcc_stream = typeid_cast *>(input_stream.get()); input_stream->readPrefix(); @@ -137,8 +137,12 @@ StableValueSpacePtr createNewStable(DMContext & context, auto delegate = context.path_pool.getStableDiskDelegator(); auto store_path = delegate.choosePath(); + DMFileBlockOutputStream::Flags flags; + flags.setRateLimit(need_rate_limit); + flags.setSingleFile(context.db_context.getSettingsRef().dt_enable_single_file_mode_dmfile); + PageId dmfile_id = context.storage_pool.newDataPageId(); - auto dmfile = writeIntoNewDMFile(context, schema_snap, input_stream, dmfile_id, store_path, need_rate_limit); + auto dmfile = writeIntoNewDMFile(context, schema_snap, input_stream, dmfile_id, store_path, flags); auto stable = std::make_shared(stable_id); stable->setFiles({dmfile}, RowKeyRange::newAll(context.is_common_handle, context.rowkey_column_size)); stable->saveMeta(wbs.meta); @@ -224,16 +228,14 @@ SegmentPtr Segment::restoreSegment(DMContext & context, PageId segment_id) switch (version) { - case SegmentFormat::V1: - { + case SegmentFormat::V1: { HandleRange range; readIntBinary(range.start, buf); readIntBinary(range.end, buf); rowkey_range = RowKeyRange::fromHandleRange(range); break; } - case SegmentFormat::V2: - { + case SegmentFormat::V2: { rowkey_range = RowKeyRange::deserialize(buf); break; } @@ -312,12 +314,12 @@ bool Segment::write(DMContext & dm_context, const RowKeyRange & delete_range) return delta->appendDeleteRange(dm_context, delete_range); } -bool Segment::writeRegionSnapshot(DMContext & dm_context, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range) +bool Segment::ingestPacks(DMContext & dm_context, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range) { auto new_range = range.shrink(rowkey_range); LOG_TRACE(log, "Segment [" << segment_id << "] write region snapshot: " << new_range.toDebugString()); - return delta->appendRegionSnapshot(dm_context, range, packs, clear_data_in_range); + return delta->ingestPacks(dm_context, range, packs, clear_data_in_range); } SegmentSnapshotPtr Segment::createSnapshot(const DMContext & dm_context, bool for_update) const @@ -454,7 +456,7 @@ BlockInputStreamPtr Segment::getInputStreamForDataExport(const DMContext & data_stream = std::make_shared>(data_stream, data_range, 0); if (reorgnize_block) { - data_stream = std::make_shared(data_stream, EXTRA_HANDLE_COLUMN_NAME); + data_stream = std::make_shared>(data_stream, EXTRA_HANDLE_COLUMN_ID, is_common_handle); } data_stream = std::make_shared>( data_stream, *read_info.read_columns, dm_context.min_version, is_common_handle); @@ -465,7 +467,6 @@ BlockInputStreamPtr Segment::getInputStreamForDataExport(const DMContext & BlockInputStreamPtr Segment::getInputStreamRaw(const DMContext & dm_context, const ColumnDefines & columns_to_read, const SegmentSnapshotPtr & segment_snap, - bool do_range_filter, size_t expected_block_size) { @@ -915,7 +916,7 @@ std::optional Segment::prepareSplitPhysical(DMContext & my_data = std::make_shared>(my_data, my_range, 0); - my_data = std::make_shared(my_data, EXTRA_HANDLE_COLUMN_NAME); + my_data = std::make_shared>(my_data, EXTRA_HANDLE_COLUMN_ID, is_common_handle); my_data = std::make_shared>( my_data, *read_info.read_columns, dm_context.min_version, is_common_handle); auto my_stable_id = segment_snap->stable->getId(); @@ -942,7 +943,7 @@ std::optional Segment::prepareSplitPhysical(DMContext & other_data = std::make_shared>(other_data, other_range, 0); - other_data = std::make_shared(other_data, EXTRA_HANDLE_COLUMN_NAME); + other_data = std::make_shared>(other_data, EXTRA_HANDLE_COLUMN_ID, is_common_handle); other_data = std::make_shared>( other_data, *read_info.read_columns, dm_context.min_version, is_common_handle); auto other_stable_id = dm_context.storage_pool.newMetaPageId(); @@ -1072,7 +1073,7 @@ StableValueSpacePtr Segment::prepareMerge(DMContext & dm_context, dm_context.stable_pack_rows); stream = std::make_shared>(stream, segment->rowkey_range, 0); - stream = std::make_shared(stream, EXTRA_HANDLE_COLUMN_NAME); + stream = std::make_shared>(stream, EXTRA_HANDLE_COLUMN_ID, dm_context.is_common_handle); stream = std::make_shared>( stream, *read_info.read_columns, dm_context.min_version, dm_context.is_common_handle); @@ -1458,7 +1459,7 @@ bool Segment::placeDelete(const DMContext & dm_context, { RowKeyValueRef first_rowkey = RowKeyColumnContainer(block.getByPosition(0).column, is_common_handle).getRowKeyValue(0); auto place_handle_range = skippable_place ? RowKeyRange::startFrom(first_rowkey, is_common_handle, rowkey_column_size) - : RowKeyRange::newAll(is_common_handle, rowkey_column_size); + : RowKeyRange::newAll(is_common_handle, rowkey_column_size); auto compacted_index = update_delta_tree.getCompactedEntries(); diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index 639ef937a0e..8367069359f 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -118,7 +118,7 @@ class Segment : private boost::noncopyable bool writeToCache(DMContext & dm_context, const Block & block, size_t offset, size_t limit); bool write(DMContext & dm_context, const Block & block); // For test only bool write(DMContext & dm_context, const RowKeyRange & delete_range); - bool writeRegionSnapshot(DMContext & dm_context, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range); + bool ingestPacks(DMContext & dm_context, const RowKeyRange & range, const DeltaPacks & packs, bool clear_data_in_range); SegmentSnapshotPtr createSnapshot(const DMContext & dm_context, bool for_update = false) const; @@ -332,12 +332,5 @@ class Segment : private boost::noncopyable Logger * log; }; -DMFilePtr writeIntoNewDMFile(DMContext & dm_context, // - const ColumnDefinesPtr & schema_snap, - const BlockInputStreamPtr & input_stream, - UInt64 file_id, - const String & parent_path, - bool need_rate_limit); - } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 9045cc1fc6b..ece6d2e8bb3 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -196,7 +196,6 @@ void StableValueSpace::calculateStableProperty(const DMContext & context, const IdSetPtr{}, UINT64_MAX, // because we just read one pack at a time true); - data_stream = std::make_shared>(data_stream, rowkey_range, 0); auto mvcc_stream = std::make_shared>( data_stream, read_columns, 0, is_common_handle); mvcc_stream->readPrefix(); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.h b/dbms/src/Storages/DeltaMerge/StoragePool.h index 6b7d3a81de0..723552519d1 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.h +++ b/dbms/src/Storages/DeltaMerge/StoragePool.h @@ -59,7 +59,7 @@ class StoragePool : private boost::noncopyable std::mutex mutex; }; -struct StorageSnapshot +struct StorageSnapshot : private boost::noncopyable { StorageSnapshot(StoragePool & storage, bool snapshot_read = true) : log_reader(storage.log(), snapshot_read ? storage.log().getSnapshot() : nullptr), diff --git a/dbms/src/Storages/DeltaMerge/tests/dm_basic_include.h b/dbms/src/Storages/DeltaMerge/tests/dm_basic_include.h index 96a85470807..5ea518b0454 100644 --- a/dbms/src/Storages/DeltaMerge/tests/dm_basic_include.h +++ b/dbms/src/Storages/DeltaMerge/tests/dm_basic_include.h @@ -111,14 +111,7 @@ class DMTestEnv } else { - if (!reversed) - { - field = Int64(beg + i); - } - else - { - field = Int64(end - 1 - i); - } + field = reversed ? Int64(end - 1 - i) : Int64(beg + i); } m_col->insert(field); } @@ -158,16 +151,17 @@ class DMTestEnv * Create a simple block with 3 columns: * * `pk` - Int64 / `version` / `tag` * @param pk `pk`'s value - * @param tso_beg `tso`'s value begin - * @param tso_end `tso`'s value end (not included) + * @param ts_beg `timestamp`'s value begin + * @param ts_end `timestamp`'s value end (not included) + * @param reversed increasing/decreasing insert `timestamp`'s value * @return */ - static Block prepareBlockWithIncreasingTso(Int64 pk, size_t tso_beg, size_t tso_end) + static Block prepareBlockWithTso(Int64 pk, size_t ts_beg, size_t ts_end, bool reversed = false) { Block block; - const size_t num_rows = (tso_end - tso_beg); + const size_t num_rows = (ts_end - ts_beg); { - ColumnWithTypeAndName col1(std::make_shared(), pk_name); + ColumnWithTypeAndName col1(nullptr, std::make_shared(), pk_name, EXTRA_HANDLE_COLUMN_ID); { IColumn::MutablePtr m_col = col1.type->createColumn(); // insert form large to small @@ -180,19 +174,19 @@ class DMTestEnv } block.insert(col1); - ColumnWithTypeAndName version_col(VERSION_COLUMN_TYPE, VERSION_COLUMN_NAME); + ColumnWithTypeAndName version_col(nullptr, VERSION_COLUMN_TYPE, VERSION_COLUMN_NAME, VERSION_COLUMN_ID); { IColumn::MutablePtr m_col = version_col.type->createColumn(); for (size_t i = 0; i < num_rows; ++i) { - Field field = Int64(tso_beg + i); + Field field = reversed ? Int64(ts_end - 1 - i) : Int64(ts_beg + i); m_col->insert(field); } version_col.column = std::move(m_col); } block.insert(version_col); - ColumnWithTypeAndName tag_col(TAG_COLUMN_TYPE, TAG_COLUMN_NAME); + ColumnWithTypeAndName tag_col(nullptr, TAG_COLUMN_TYPE, TAG_COLUMN_NAME, TAG_COLUMN_ID); { IColumn::MutablePtr m_col = tag_col.type->createColumn(); auto & column_data = typeid_cast &>(*m_col).getData(); @@ -216,7 +210,10 @@ class DMTestEnv Block block; const size_t num_rows = 1; { - ColumnWithTypeAndName col1(is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, pk_name); + ColumnWithTypeAndName col1(nullptr, + is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, + pk_name, + EXTRA_HANDLE_COLUMN_ID); { IColumn::MutablePtr m_col = col1.type->createColumn(); // insert form large to small @@ -238,7 +235,7 @@ class DMTestEnv } block.insert(col1); - ColumnWithTypeAndName version_col(VERSION_COLUMN_TYPE, VERSION_COLUMN_NAME); + ColumnWithTypeAndName version_col(nullptr, VERSION_COLUMN_TYPE, VERSION_COLUMN_NAME, VERSION_COLUMN_ID); { IColumn::MutablePtr m_col = version_col.type->createColumn(); m_col->insert(tso); @@ -246,7 +243,7 @@ class DMTestEnv } block.insert(version_col); - ColumnWithTypeAndName tag_col(TAG_COLUMN_TYPE, TAG_COLUMN_NAME); + ColumnWithTypeAndName tag_col(nullptr, TAG_COLUMN_TYPE, TAG_COLUMN_NAME, TAG_COLUMN_ID); { IColumn::MutablePtr m_col = tag_col.type->createColumn(); auto & column_data = typeid_cast &>(*m_col).getData(); @@ -256,7 +253,7 @@ class DMTestEnv } block.insert(tag_col); - ColumnWithTypeAndName str_col(DataTypeFactory::instance().get("String"), colname); + ColumnWithTypeAndName str_col(nullptr, DataTypeFactory::instance().get("String"), colname); { IColumn::MutablePtr m_col = str_col.type->createColumn(); m_col->insert(value); @@ -330,6 +327,12 @@ class DMTestEnv } return block; } + + static int getPseudoRandomNumber() + { + static int num = 0; + return num++; + } }; } // namespace tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp new file mode 100644 index 00000000000..3169aa8826b --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace DM +{ +namespace tests +{ + +TEST(PKSquash_test, WithExtraSort) +{ + BlocksList blocks; + + size_t rows_per_block = 10; + size_t num_rows_write = 0; + { + // pk asc, ts desc + blocks.push_back(DMTestEnv::prepareBlockWithTso(4, 10000 + rows_per_block * 2, 10000 + rows_per_block * 3, true)); + num_rows_write += blocks.back().rows(); + blocks.push_back(DMTestEnv::prepareBlockWithTso(4, 10000 + rows_per_block, 10000 + rows_per_block * 2, true)); + num_rows_write += blocks.back().rows(); + blocks.push_back(DMTestEnv::prepareBlockWithTso(4, 10000, 10000 + rows_per_block, true)); + num_rows_write += blocks.back().rows(); + + { + Block mix_pks_block = DMTestEnv::prepareBlockWithTso(5, 10000, 10000 + rows_per_block, true); + Block b2 = DMTestEnv::prepareBlockWithTso(6, 10000 + rows_per_block, 10000 + rows_per_block * 2, true); + concat(mix_pks_block, b2); + blocks.push_back(mix_pks_block); + num_rows_write += blocks.back().rows(); + } + { + Block mix_pks_block = DMTestEnv::prepareBlockWithTso(6, 10000, 10000 + rows_per_block, true); + Block b2 = DMTestEnv::prepareBlockWithTso(7, 10000 + rows_per_block, 10000 + rows_per_block * 2, true); + concat(mix_pks_block, b2); + blocks.push_back(mix_pks_block); + num_rows_write += blocks.back().rows(); + } + blocks.push_back(DMTestEnv::prepareBlockWithTso(7, 10000, 10000 + rows_per_block, true)); + num_rows_write += blocks.back().rows(); + } + + // Sorted by pk, tso asc + SortDescription sort // + = SortDescription{// + SortColumnDescription{EXTRA_HANDLE_COLUMN_NAME, 1, 0}, + SortColumnDescription{VERSION_COLUMN_NAME, 1, 0}}; + + { + auto in = std::make_shared>( + std::make_shared(blocks.begin(), blocks.end()), TiDBPkColumnID, false); + size_t num_blocks_read = 0; + size_t num_rows_read = 0; + in->readPrefix(); + Block block; + while (true) + { + block = in->read(); + if (!block) + break; + + num_blocks_read += 1; + if (num_blocks_read == 1) + { + // for pk == 4 + EXPECT_EQ(block.rows(), rows_per_block * 3); + } + else if (num_blocks_read == 2) + { + // for pk in (5, 6) + EXPECT_EQ(block.rows(), rows_per_block * 3); + } + else + { + // for pk == 7 + EXPECT_EQ(block.rows(), rows_per_block * 2); + } + num_rows_read += block.rows(); + // Should be sorted + ASSERT_TRUE(isAlreadySorted(block, sort)); + } + in->readSuffix(); + ASSERT_EQ(num_blocks_read, 3); + ASSERT_EQ(num_rows_read, num_rows_write); + } +} + +} // namespace tests +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_index_manager.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_index_manager.cpp index 1b3c26f4642..5cd1f63e413 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_index_manager.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_index_manager.cpp @@ -1,6 +1,5 @@ #include #include -#include #include namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index 4f1b4f5f541..371da5ee020 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -4,12 +4,15 @@ #include #include #include -#include + +#define private public #include +#undef private +#include +#include #include #include #include -#include #include @@ -28,6 +31,12 @@ extern const char force_triggle_foreground_flush[]; namespace DM { +extern DMFilePtr writeIntoNewDMFile(DMContext & dm_context, // + const ColumnDefinesPtr & schema_snap, + const BlockInputStreamPtr & input_stream, + UInt64 file_id, + const String & parent_path, + DMFileBlockOutputStream::Flags flags); namespace tests { @@ -111,14 +120,17 @@ class DeltaMergeStore_test : public ::testing::Test, public testing::WithParamIn std::pair> genDMFile(DMContext & context, const Block & block) { - auto file_id = context.storage_pool.newDataPageId(); - auto input_stream = std::make_shared(block); - auto delegate = context.path_pool.getStableDiskDelegator(); - auto store_path = delegate.choosePath(); - auto dmfile = writeIntoNewDMFile( - context, std::make_shared(store->getTableColumns()), input_stream, file_id, store_path, false); + auto input_stream = std::make_shared(block); + auto [store_path, file_id] = store->preAllocateIngestFile(); + + DMFileBlockOutputStream::Flags flags; + flags.setSingleFile(DMTestEnv::getPseudoRandomNumber() % 2); + + auto dmfile = writeIntoNewDMFile( + context, std::make_shared(store->getTableColumns()), input_stream, file_id, store_path, flags); + - delegate.addDTFile(file_id, dmfile->getBytesOnDisk(), store_path); + store->preIngestFile(store_path, file_id, dmfile->getBytesOnDisk()); auto & pk_column = block.getByPosition(0).column; auto min_pk = pk_column->getInt(0); @@ -254,12 +266,12 @@ try { case TestMode::V1_BlockOnly: case TestMode::V2_BlockOnly: - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); break; default: { auto dm_context = store->newDMContext(*context, context->getSettingsRef()); auto [range, file_ids] = genDMFile(*dm_context, block); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); break; } } @@ -364,12 +376,12 @@ try { case TestMode::V1_BlockOnly: case TestMode::V2_BlockOnly: - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); break; default: { auto dm_context = store->newDMContext(*context, context->getSettingsRef()); auto [range, file_ids] = genDMFile(*dm_context, block); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); break; } } @@ -458,9 +470,9 @@ try { case TestMode::V1_BlockOnly: case TestMode::V2_BlockOnly: { - store->write(*context, context->getSettingsRef(), block1); - store->write(*context, context->getSettingsRef(), block2); - store->write(*context, context->getSettingsRef(), block3); + store->write(*context, context->getSettingsRef(), std::move(block1)); + store->write(*context, context->getSettingsRef(), std::move(block2)); + store->write(*context, context->getSettingsRef(), std::move(block3)); break; } case TestMode::V2_FileOnly: { @@ -472,7 +484,7 @@ try auto file_ids = file_ids1; file_ids.insert(file_ids.cend(), file_ids2.begin(), file_ids2.end()); file_ids.insert(file_ids.cend(), file_ids3.begin(), file_ids3.end()); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); break; } case TestMode::V2_Mix: { @@ -482,9 +494,9 @@ try auto range = range1.merge(range3); auto file_ids = file_ids1; file_ids.insert(file_ids.cend(), file_ids3.begin(), file_ids3.end()); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); - store->write(*context, context->getSettingsRef(), block2); + store->write(*context, context->getSettingsRef(), std::move(block2)); break; } } @@ -536,9 +548,9 @@ try { case TestMode::V1_BlockOnly: case TestMode::V2_BlockOnly: { - store->write(*context, context->getSettingsRef(), block1); - store->write(*context, context->getSettingsRef(), block2); - store->write(*context, context->getSettingsRef(), block3); + store->write(*context, context->getSettingsRef(), std::move(block1)); + store->write(*context, context->getSettingsRef(), std::move(block2)); + store->write(*context, context->getSettingsRef(), std::move(block3)); break; } case TestMode::V2_FileOnly: { @@ -550,11 +562,11 @@ try auto file_ids = file_ids1; file_ids.insert(file_ids.cend(), file_ids2.begin(), file_ids2.end()); file_ids.insert(file_ids.cend(), file_ids3.begin(), file_ids3.end()); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); break; } case TestMode::V2_Mix: { - store->write(*context, context->getSettingsRef(), block2); + store->write(*context, context->getSettingsRef(), std::move(block2)); auto dm_context = store->newDMContext(*context, context->getSettingsRef()); auto [range1, file_ids1] = genDMFile(*dm_context, block1); @@ -562,7 +574,7 @@ try auto range = range1.merge(range3); auto file_ids = file_ids1; file_ids.insert(file_ids.cend(), file_ids3.begin(), file_ids3.end()); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); break; } } @@ -654,7 +666,7 @@ try { // Write 7 rows that would not trigger a split Block block = DMTestEnv::prepareSimpleWriteBlock(0, 8, false); - store->write(*context, settings, block); + store->write(*context, settings, std::move(block)); } { @@ -689,7 +701,7 @@ try { // Write rows that would trigger a split Block block = DMTestEnv::prepareSimpleWriteBlock(8, 9, false); - store->write(*context, settings, block); + store->write(*context, settings, std::move(block)); } // Now there is 2 segments @@ -741,7 +753,7 @@ try { // write to store Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_tso1, false, tso1); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } const UInt64 tso2 = 890; @@ -749,7 +761,7 @@ try { // write to store Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_tso1, num_rows_tso1 + num_rows_tso2, false, tso2); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -842,6 +854,274 @@ try } CATCH +TEST_P(DeltaMergeStore_test, Ingest) +try +{ + if (mode == TestMode::V1_BlockOnly) + return; + + const UInt64 tso1 = 4; + const size_t num_rows_before_ingest = 128; + // Write to store [0, 128) + { + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_before_ingest, false, tso1); + store->write(*context, context->getSettingsRef(), std::move(block)); + } + + const UInt64 tso2 = 10; + const UInt64 tso3 = 18; + + { + // Prepare DTFiles for ingesting + auto dm_context = store->newDMContext(*context, context->getSettingsRef()); + + auto [range1, file_ids1] = genDMFile(*dm_context, DMTestEnv::prepareSimpleWriteBlock(32, 48, false, tso2)); + auto [range2, file_ids2] = genDMFile(*dm_context, DMTestEnv::prepareSimpleWriteBlock(80, 256, false, tso3)); + + auto file_ids = file_ids1; + file_ids.insert(file_ids.cend(), file_ids2.begin(), file_ids2.end()); + auto ingest_range = RowKeyRange::fromHandleRange(HandleRange{32, 256}); + // verify that ingest_range must not less than range1.merge(range2) + ASSERT_ROWKEY_RANGE_EQ(ingest_range, range1.merge(range2).merge(ingest_range)); + + store->ingestFiles(dm_context, ingest_range, file_ids, /*clear_data_in_range*/ true); + } + + + // After ingesting, the data in [32, 128) should be overwrite by the data in ingested files. + { + // Read all data <= tso1 + // We can only get [0, 32) with tso1 + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso1, + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + Int64 expect_pk = 0; + UInt64 expect_tso = tso1; + while (Block block = in->read()) + { + ASSERT_TRUE(block.has(DMTestEnv::pk_name)); + ASSERT_TRUE(block.has(VERSION_COLUMN_NAME)); + auto pk_c = block.getByName(DMTestEnv::pk_name); + auto v_c = block.getByName(VERSION_COLUMN_NAME); + for (size_t i = 0; i < block.rows(); ++i) + { + // std::cerr << "pk:" << pk_c.column->getInt(i) << ", ver:" << v_c.column->getInt(i) << std::endl; + ASSERT_EQ(pk_c.column->getInt(i), expect_pk++); + ASSERT_EQ(v_c.column->getUInt(i), expect_tso); + } + num_rows_read += block.rows(); + } + in->readSuffix(); + EXPECT_EQ(num_rows_read, 32UL) << "Data [32, 128) before ingest should be erased, should only get [0, 32)"; + } + + { + // Read all data between [tso, tso2) + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso2 - 1, + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + Int64 expect_pk = 0; + UInt64 expect_tso = tso1; + while (Block block = in->read()) + { + ASSERT_TRUE(block.has(DMTestEnv::pk_name)); + ASSERT_TRUE(block.has(VERSION_COLUMN_NAME)); + auto pk_c = block.getByName(DMTestEnv::pk_name); + auto v_c = block.getByName(VERSION_COLUMN_NAME); + for (size_t i = 0; i < block.rows(); ++i) + { + // std::cerr << "pk:" << pk_c.column->getInt(i) << ", ver:" << v_c.column->getInt(i) << std::endl; + ASSERT_EQ(pk_c.column->getInt(i), expect_pk++); + ASSERT_EQ(v_c.column->getUInt(i), expect_tso); + } + num_rows_read += block.rows(); + } + in->readSuffix(); + EXPECT_EQ(num_rows_read, 32UL) << "Data [32, 128) after ingest with tso less than: " << tso2 + << " are erased, should only get [0, 32)"; + } + + { + // Read all data between [tso2, tso3) + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso3 - 1, + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + { + num_rows_read += block.rows(); + } + in->readSuffix(); + EXPECT_EQ(num_rows_read, 32UL + 16) << "The rows number after ingest with tso less than " << tso3 << " is not match"; + } + + { + // Read all data between [tso2, tso3) + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + num_rows_read += block.rows(); + in->readSuffix(); + EXPECT_EQ(num_rows_read, 32UL + (48 - 32) + (256UL - 80)) << "The rows number after ingest is not match"; + } + + { + // Read with two point get, issue 1616 + auto range0 = RowKeyRange::fromHandleRange(HandleRange(32, 33)); + auto range1 = RowKeyRange::fromHandleRange(HandleRange(40, 41)); + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {range0, range1}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + num_rows_read += block.rows(); + in->readSuffix(); + EXPECT_EQ(num_rows_read, 2UL) << "The rows number of two point get is not match"; + } +} +CATCH + +TEST_P(DeltaMergeStore_test, IngestEmptyFileLists) +try +{ + if (mode == TestMode::V1_BlockOnly) + return; + + const UInt64 tso1 = 4; + const size_t num_rows_before_ingest = 128; + // Write to store [0, 128) + { + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_before_ingest, false, tso1); + store->write(*context, context->getSettingsRef(), std::move(block)); + } + + // Test that if we ingest a empty file list, the data in range will be removed. + // The ingest range is [32, 256) + { + auto dm_context = store->newDMContext(*context, context->getSettingsRef()); + + std::vector file_ids ; + auto ingest_range = RowKeyRange::fromHandleRange(HandleRange{32, 256}); + store->ingestFiles(dm_context, ingest_range, file_ids, /*clear_data_in_range*/ true); + } + + + // After ingesting, the data in [32, 128) should be overwrite by the data in ingested files. + { + // Read all data <= tso1 + // We can only get [0, 32) with tso1 + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ tso1, + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + Int64 expect_pk = 0; + UInt64 expect_tso = tso1; + while (Block block = in->read()) + { + ASSERT_TRUE(block.has(DMTestEnv::pk_name)); + ASSERT_TRUE(block.has(VERSION_COLUMN_NAME)); + auto pk_c = block.getByName(DMTestEnv::pk_name); + auto v_c = block.getByName(VERSION_COLUMN_NAME); + for (size_t i = 0; i < block.rows(); ++i) + { + // std::cerr << "pk:" << pk_c.column->getInt(i) << ", ver:" << v_c.column->getInt(i) << std::endl; + ASSERT_EQ(pk_c.column->getInt(i), expect_pk++); + ASSERT_EQ(v_c.column->getUInt(i), expect_tso); + } + num_rows_read += block.rows(); + } + in->readSuffix(); + EXPECT_EQ(num_rows_read, 32UL) << "Data [32, 128) before ingest should be erased, should only get [0, 32)"; + } + + { + // Read all data + const auto & columns = store->getTableColumns(); + BlockInputStreams ins = store->read(*context, + context->getSettingsRef(), + columns, + {RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + EMPTY_FILTER, + /* expected_block_size= */ 1024); + ASSERT_EQ(ins.size(), 1UL); + BlockInputStreamPtr in = ins[0]; + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + num_rows_read += block.rows(); + in->readSuffix(); + EXPECT_EQ(num_rows_read, 32UL) << "The rows number after ingest is not match"; + } +} +CATCH + TEST_P(DeltaMergeStore_test, Split) try { @@ -867,14 +1147,14 @@ try auto write_as_file = [&]() { auto dm_context = store->newDMContext(*context, context->getSettingsRef()); auto [range, file_ids] = genDMFile(*dm_context, block); - store->writeRegionSnapshot(dm_context, range, file_ids, false); + store->ingestFiles(dm_context, range, file_ids, false); }; switch (mode) { case TestMode::V1_BlockOnly: case TestMode::V2_BlockOnly: - store->write(*context, settings, block); + store->write(*context, settings, std::move(block)); break; case TestMode::V2_FileOnly: write_as_file(); @@ -882,7 +1162,7 @@ try case TestMode::V2_Mix: { if ((std::rand() % 2) == 0) { - store->write(*context, settings, block); + store->write(*context, settings, std::move(block)); } else { @@ -1000,7 +1280,7 @@ try } block.insert(col2); } - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -1113,7 +1393,7 @@ try } block.insert(col2); } - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -1208,7 +1488,7 @@ try } block.insert(col1); } - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -1298,7 +1578,7 @@ try size_t num_rows_write = 1; { Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } // DDL add column f32 with default value @@ -1372,7 +1652,7 @@ try // write some rows before DDL { Block block = DMTestEnv::prepareSimpleWriteBlock(0, 1, false); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } // DDL add column date with default value @@ -1477,7 +1757,7 @@ try } block.insert(col2); } - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -1587,7 +1867,7 @@ try { // write to store Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false, /*tso=*/2, col_name_before_ddl, col_id_ddl, col_type); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -1674,7 +1954,7 @@ try // Then write new block with new pk name Block block = DMTestEnv::prepareSimpleWriteBlock( num_rows_write, num_rows_write * 2, false, /*tso=*/2, col_name_after_ddl, col_id_ddl, col_type); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { // read all columns from store @@ -1745,7 +2025,7 @@ try FailPointHelper::enableFailPoint(FailPoints::force_triggle_background_merge_delta); Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } // DDL add column f32 with default value @@ -1824,7 +2104,7 @@ try f_col.column = std::move(m_col); } block.insert(f_col); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } // disable pause so that delta-merge can continue @@ -1952,7 +2232,7 @@ try } block.insert(std::move(i8)); } - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } { @@ -2082,9 +2362,9 @@ try EXTRA_HANDLE_COLUMN_STRING_TYPE, true, rowkey_column_size); - store->write(*context, context->getSettingsRef(), block1); - store->write(*context, context->getSettingsRef(), block2); - store->write(*context, context->getSettingsRef(), block3); + store->write(*context, context->getSettingsRef(), std::move(block1)); + store->write(*context, context->getSettingsRef(), std::move(block2)); + store->write(*context, context->getSettingsRef(), std::move(block3)); store->flushCache(*context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); } @@ -2152,9 +2432,9 @@ try EXTRA_HANDLE_COLUMN_STRING_TYPE, true, rowkey_column_size); - store->write(*context, context->getSettingsRef(), block1); - store->write(*context, context->getSettingsRef(), block2); - store->write(*context, context->getSettingsRef(), block3); + store->write(*context, context->getSettingsRef(), std::move(block1)); + store->write(*context, context->getSettingsRef(), std::move(block2)); + store->write(*context, context->getSettingsRef(), std::move(block3)); store->flushCache(*context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); } @@ -2238,7 +2518,7 @@ try Block block = DMTestEnv::prepareSimpleWriteBlock( 0, 128, false, 2, EXTRA_HANDLE_COLUMN_NAME, EXTRA_HANDLE_COLUMN_ID, EXTRA_HANDLE_COLUMN_STRING_TYPE, true, rowkey_column_size); - store->write(*context, context->getSettingsRef(), block); + store->write(*context, context->getSettingsRef(), std::move(block)); } // Test Reading first { diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp index 9bc78feadd6..def30faa33f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp @@ -22,6 +22,25 @@ namespace DM namespace tests { +TEST(DMFileWriterFlags_test, SetClearFlags) +{ + using Flags = DMFileWriter::Flags; + + Flags flags; + + bool f = false; + flags.setRateLimit(f); + EXPECT_FALSE(flags.needRateLimit()); + flags.setSingleFile(f); + EXPECT_FALSE(flags.isSingleFile()); + + f = true; + flags.setRateLimit(f); + EXPECT_TRUE(flags.needRateLimit()); + flags.setSingleFile(f); + EXPECT_TRUE(flags.isSingleFile()); +} + String paramToString(const ::testing::TestParamInfo & info) { const auto mode = info.param; @@ -153,6 +172,8 @@ try stream->write(block1, block_property1); stream->write(block2, block_property2); stream->writeSuffix(); + + ASSERT_EQ(dm_file->getPackProperties().property_size(), 2); } @@ -189,10 +210,7 @@ try /// Test restore the file from disk and read { - auto id = dm_file->fileId(); - dm_file.reset(); - auto file_provider = dbContext().getFileProvider(); - dm_file = DMFile::restore(file_provider, id, 0, parent_path, /*read_meta=*/true); + dm_file = restoreDMFile(); // Test dt property read success auto propertys = dm_file->getPackProperties(); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp index b53d2b1f046..a79d30373d4 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -78,7 +78,7 @@ bool checkMatch(const String & test_case, // DeltaMergeStorePtr store = std::make_shared( context, false, "test_database", "test_table", table_columns, getExtraHandleColumnDefine(false), false, 1); - store->write(context, context.getSettingsRef(), block); + store->write(context, context.getSettingsRef(), std::move(block)); store->flushCache(context, all_range); store->mergeDeltaAll(context); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_region.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_region.cpp index 19b7286d1d6..0513b27c12a 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_region.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_region.cpp @@ -109,7 +109,7 @@ try size_t step = rand() % 1000; LOG_DEBUG(log, "step " << step); auto block = DMTestEnv::prepareBlockWithIncreasingPKAndTs(step, cur_rows, cur_rows); - store->write(*context, settings, block); + store->write(*context, settings, std::move(block)); cur_rows += step; check_exact(RowKeyRange::fromHandleRange({0, (Int64)(cur_rows)}), cur_rows, cur_rows * bytes_per_rows); @@ -184,7 +184,7 @@ try size_t step = rand() % 1000; LOG_DEBUG(log, "step " << step); auto block = DMTestEnv::prepareBlockWithIncreasingPKAndTs(step, cur_rows, cur_rows); - store->write(*context, settings, block); + store->write(*context, settings, std::move(block)); cur_rows += step; check_split_point(random_range(cur_rows)); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp index a0d8ba57bb8..a390f4d4c4d 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,12 @@ namespace DB { namespace DM { +extern DMFilePtr writeIntoNewDMFile(DMContext & dm_context, // + const ColumnDefinesPtr & schema_snap, + const BlockInputStreamPtr & input_stream, + UInt64 file_id, + const String & parent_path, + DMFileBlockOutputStream::Flags flags); namespace tests { @@ -177,7 +184,6 @@ try { // flush segment segment = segment->mergeDelta(dmContext(), tableColumns()); - ; } { @@ -1035,8 +1041,12 @@ class Segment_test_2 : public Segment_test, public testing::WithParamInterface(block); auto delegate = context.path_pool.getStableDiskDelegator(); auto store_path = delegate.choosePath(); + + DMFileBlockOutputStream::Flags flags; + flags.setSingleFile(DMTestEnv::getPseudoRandomNumber() % 2); + auto dmfile - = writeIntoNewDMFile(context, std::make_shared(*tableColumns()), input_stream, file_id, store_path, false); + = writeIntoNewDMFile(context, std::make_shared(*tableColumns()), input_stream, file_id, store_path, flags); delegate.addDTFile(file_id, dmfile->getBytesOnDisk(), store_path); @@ -1066,8 +1076,7 @@ try case Segment_test_Mode::V2_BlockOnly: segment->write(dmContext(), std::move(block)); break; - case Segment_test_Mode::V2_FileOnly: - { + case Segment_test_Mode::V2_FileOnly: { auto delegate = dmContext().path_pool.getStableDiskDelegator(); auto file_provider = dmContext().db_context.getFileProvider(); auto [range, file_ids] = genDMFile(dmContext(), block); @@ -1080,7 +1089,7 @@ try wbs.data.putExternal(file_id, 0); wbs.writeLogAndData(); - segment->writeRegionSnapshot(dmContext(), range, {pack}, false); + segment->ingestPacks(dmContext(), range, {pack}, false); break; } default: @@ -1343,7 +1352,6 @@ try { segment->flushCache(dmContext()); segment = segment->mergeDelta(dmContext(), tableColumns()); - ; } { @@ -1521,7 +1529,6 @@ try { segment->flushCache(dmContext()); segment = segment->mergeDelta(dmContext(), tableColumns()); - ; } { 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 6633e337b30..d052669bb7f 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 @@ -24,7 +24,6 @@ #include #include #include -#include #include @@ -297,7 +296,6 @@ try { path.remove(true); } - std::cout << "TemporaryPath " << path_name << std::endl; // primary_expr_ast ASTPtr astptr(new ASTIdentifier(table_name, ASTIdentifier::Kind::Table)); diff --git a/dbms/src/Storages/FormatVersion.h b/dbms/src/Storages/FormatVersion.h index 97afde98a74..292bfb0c67d 100644 --- a/dbms/src/Storages/FormatVersion.h +++ b/dbms/src/Storages/FormatVersion.h @@ -77,7 +77,7 @@ inline static const StorageFormatVersion STORAGE_FORMAT_V2 = StorageFormatVersio .page = PageFormat::V2, }; -inline StorageFormatVersion STORAGE_FORMAT_CURRENT = STORAGE_FORMAT_V1; +inline StorageFormatVersion STORAGE_FORMAT_CURRENT = STORAGE_FORMAT_V2; inline const StorageFormatVersion & toStorageFormat(UInt64 setting) { @@ -96,4 +96,4 @@ inline void setStorageFormat(UInt64 setting) { STORAGE_FORMAT_CURRENT = toStorag inline void setStorageFormat(const StorageFormatVersion & version) { STORAGE_FORMAT_CURRENT = version; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/Page/PageStorage.h b/dbms/src/Storages/Page/PageStorage.h index ca3e41ce0dd..10bcbb28bd6 100644 --- a/dbms/src/Storages/Page/PageStorage.h +++ b/dbms/src/Storages/Page/PageStorage.h @@ -36,7 +36,7 @@ using PSDiskDelegatorPtr = std::shared_ptr; * * This class is multi-threads safe. Support single thread write, and multi threads read. */ -class PageStorage +class PageStorage : private boost::noncopyable { public: struct Config @@ -222,7 +222,7 @@ class PageStorage TiFlashMetricsPtr metrics; }; -class PageReader +class PageReader : private boost::noncopyable { public: /// Not snapshot read. diff --git a/dbms/src/Storages/Page/tests/gtest_page_map_version_set.cpp b/dbms/src/Storages/Page/tests/gtest_page_map_version_set.cpp index 3e97efc4ba9..4df1a45f136 100644 --- a/dbms/src/Storages/Page/tests/gtest_page_map_version_set.cpp +++ b/dbms/src/Storages/Page/tests/gtest_page_map_version_set.cpp @@ -1,6 +1,3 @@ -#include -#include - #include #define protected public diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 918c92f56c0..e0eda3479a4 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -397,7 +397,7 @@ class DMBlockOutputStream : public IBlockOutputStream write_block.insert(ColumnWithTypeAndName(std::move(col), column.type, column.name, column.column_id)); } - store->write(db_context, db_settings, write_block); + store->write(db_context, db_settings, std::move(write_block)); } } } @@ -483,7 +483,7 @@ void StorageDeltaMerge::write(Block && block, const Settings & settings) FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_write_to_storage); - store->write(global_context, settings, block); + store->write(global_context, settings, std::move(block)); } std::unordered_set parseSegmentSet(const ASTPtr & ast) @@ -711,6 +711,14 @@ void StorageDeltaMerge::deleteRange(const DM::RowKeyRange & range_to_delete, con return getAndMaybeInitStore()->deleteRange(global_context, settings, range_to_delete); } +void StorageDeltaMerge::ingestFiles( + const DM::RowKeyRange & range, const std::vector & file_ids, bool clear_data_in_range, const Settings & settings) +{ + auto metrics = global_context.getTiFlashMetrics(); + GET_METRIC(metrics, tiflash_storage_command_count, type_ingest).Increment(); + return getAndMaybeInitStore()->ingestFiles(global_context, settings, range, file_ids, clear_data_in_range); +} + UInt64 StorageDeltaMerge::onSyncGc(Int64 limit) { if (store_inited.load(std::memory_order_acquire)) @@ -984,6 +992,7 @@ void StorageDeltaMerge::rename( return; } + /// Note that this routine is only left for CI tests. `clean_rename` should always be true in production env. auto & store = getAndMaybeInitStore(); // For DatabaseOrdinary, we need to rename data path, then recreate a new store. const String new_path = new_path_to_db + "/" + new_table_name; diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 15858efcacd..8fb8d697d0b 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -68,6 +68,9 @@ class StorageDeltaMerge : public ext::shared_ptr_helper, publ void deleteRange(const DM::RowKeyRange & range_to_delete, const Settings & settings); + void ingestFiles( + const DM::RowKeyRange & range, const std::vector & file_ids, bool clear_data_in_range, const Settings & settings); + UInt64 onSyncGc(Int64) override; void rename(const String & new_path_to_db, @@ -147,7 +150,6 @@ class StorageDeltaMerge : public ext::shared_ptr_helper, publ bool dataDirExist(); private: - using ColumnIdMap = std::unordered_map; struct TableColumnInfo { TableColumnInfo(const String& db, const String& table, const ASTPtr& pk) diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 40ecdcc6be0..08eff6d9687 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -11,7 +13,9 @@ #include #include #include +#include #include +#include #include #include @@ -19,17 +23,24 @@ namespace DB { +std::tuple, bool, DM::ColumnDefinesPtr> // +AtomicGetStorageSchema(const RegionPtr & region, TMTContext & tmt); + namespace FailPoints { +extern const char force_set_sst_to_dtfile_block_size[]; +extern const char force_set_sst_decode_rand[]; extern const char pause_until_apply_raft_snapshot[]; } // namespace FailPoints namespace ErrorCodes { extern const int LOGICAL_ERROR; -} +extern const int REGION_DATA_SCHEMA_UPDATED; +} // namespace ErrorCodes -void KVStore::checkAndApplySnapshot(const RegionPtrWithBlock & new_region, TMTContext & tmt) +template +void KVStore::checkAndApplySnapshot(const RegionPtrWrap & new_region, TMTContext & tmt) { auto region_id = new_region->id(); auto old_region = getRegion(region_id); @@ -130,39 +141,39 @@ void KVStore::checkAndApplySnapshot(const RegionPtrWithBlock & new_region, TMTCo onSnapshot(new_region, old_region, old_applied_index, tmt); } -void KVStore::onSnapshot(const RegionPtrWithBlock & new_region_wrap, RegionPtr old_region, UInt64 old_region_index, TMTContext & tmt) +template +void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_region, UInt64 old_region_index, TMTContext & tmt) { RegionID region_id = new_region_wrap->id(); { auto table_id = new_region_wrap->getMappedTableID(); - if (auto storage = tmt.getStorages().get(table_id); storage) + if (auto storage = tmt.getStorages().get(table_id); storage && storage->engineType() == TiDB::StorageEngine::DT) { - switch (storage->engineType()) + try { - case TiDB::StorageEngine::DT: + auto & context = tmt.getContext(); + // acquire lock so that no other threads can drop storage + auto table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + auto dm_storage = std::dynamic_pointer_cast(storage); + auto key_range = DM::RowKeyRange::fromRegionRange( + new_region_wrap->getRange(), table_id, storage->isCommonHandle(), storage->getRowKeyColumnSize()); + if constexpr (std::is_same_v) { - try - { - auto & context = tmt.getContext(); - // acquire lock so that no other threads can drop storage - auto table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); - auto dm_storage = std::dynamic_pointer_cast(storage); - auto key_range = DM::RowKeyRange::fromRegionRange( - new_region_wrap->getRange(), table_id, storage->isCommonHandle(), storage->getRowKeyColumnSize()); - // Call `deleteRange` to delete data for range - dm_storage->deleteRange(key_range, context.getSettingsRef()); - } - catch (DB::Exception & e) - { - // We can ignore if storage is dropped. - if (e.code() != ErrorCodes::TABLE_IS_DROPPED) - throw; - } - break; + // Call `ingestFiles` to delete data for range and ingest external DTFiles. + dm_storage->ingestFiles(key_range, new_region_wrap.ingest_ids, /*clear_data_in_range=*/true, context.getSettingsRef()); } - default: - break; + else + { + // Call `deleteRange` to delete data for range + dm_storage->deleteRange(key_range, context.getSettingsRef()); + } + } + catch (DB::Exception & e) + { + // We can ignore if storage is dropped. + if (e.code() != ErrorCodes::TABLE_IS_DROPPED) + throw; } } } @@ -172,20 +183,24 @@ void KVStore::onSnapshot(const RegionPtrWithBlock & new_region_wrap, RegionPtr o auto & region_table = tmt.getRegionTable(); // extend region to make sure data won't be removed. region_table.extendRegionRange(region_id, *range); - // try to flush data into ch first. - try + // For `RegionPtrWithBlock`, try to flush data into storage first. + if constexpr (std::is_same_v) { - auto tmp = region_table.tryFlushRegion(new_region_wrap, false); + try + { + auto tmp = region_table.tryFlushRegion(new_region_wrap, false); + { + std::lock_guard lock(bg_gc_region_data_mutex); + bg_gc_region_data.push_back(std::move(tmp)); + } + tryFlushRegionCacheInStorage(tmt, *new_region_wrap, log); + } + catch (...) { - std::lock_guard lock(bg_gc_region_data_mutex); - bg_gc_region_data.push_back(std::move(tmp)); + tryLogCurrentException(__PRETTY_FUNCTION__); } - tryFlushRegionCacheInStorage(tmt, *new_region_wrap, log); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); } + // For `RegionPtrWithSnapshotFiles`, don't need to flush cache. } RegionPtr new_region = new_region_wrap.base; @@ -196,7 +211,7 @@ void KVStore::onSnapshot(const RegionPtrWithBlock & new_region_wrap, RegionPtr o if (getRegion(region_id) != old_region || (old_region && old_region_index != old_region->appliedIndex())) { throw Exception( - std::string(__PRETTY_FUNCTION__) + ": region " + std::to_string(region_id) + " instance changed, should not happen", + std::string(__PRETTY_FUNCTION__) + ": region " + DB::toString(region_id) + " instance changed, should not happen", ErrorCodes::LOGICAL_ERROR); } @@ -220,10 +235,11 @@ void KVStore::onSnapshot(const RegionPtrWithBlock & new_region_wrap, RegionPtr o } } - extern RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr &, Context &); -RegionPreDecodeBlockDataPtr KVStore::preHandleSnapshot(RegionPtr new_region, const SSTViewVec snaps, TMTContext & tmt) +/// `preHandleSnapshotToBlock` read data from SSTFiles and predoced the data as a block +RegionPreDecodeBlockDataPtr KVStore::preHandleSnapshotToBlock( + RegionPtr new_region, const SSTViewVec snaps, uint64_t /*index*/, uint64_t /*term*/, TMTContext & tmt) { RegionPreDecodeBlockDataPtr cache{nullptr}; { @@ -282,7 +298,112 @@ RegionPreDecodeBlockDataPtr KVStore::preHandleSnapshot(RegionPtr new_region, con return cache; } -void KVStore::handlePreApplySnapshot(const RegionPtrWithBlock & new_region, TMTContext & tmt) +std::vector KVStore::preHandleSnapshotToFiles( + RegionPtr new_region, const SSTViewVec snaps, uint64_t index, uint64_t term, TMTContext & tmt) +{ + return preHandleSSTsToDTFiles(new_region, snaps, index, term, DM::FileConvertJobType::ApplySnapshot, tmt); +} + +/// `preHandleSSTsToDTFiles` read data from SSTFiles and generate DTFile(s) for commited data +/// return the ids of DTFile(s), the uncommited data will be inserted to `new_region` +std::vector KVStore::preHandleSSTsToDTFiles( + RegionPtr new_region, const SSTViewVec snaps, uint64_t /*index*/, uint64_t /*term*/, DM::FileConvertJobType job_type, TMTContext & tmt) +{ + auto context = tmt.getContext(); + bool force_decode = false; + size_t expected_block_size = DEFAULT_MERGE_BLOCK_SIZE; + + // Use failpoint to change the expected_block_size for some test cases + fiu_do_on(FailPoints::force_set_sst_to_dtfile_block_size, { expected_block_size = 3; }); + + PageIds ids; + while (true) + { + // If any schema changes is detected during decoding SSTs to DTFiles, we need to cancel and recreate DTFiles with + // the latest schema. Or we will get trouble in `BoundedSSTFilesToBlockInputStream`. + std::shared_ptr stream; + try + { + // Get storage schema atomically, will do schema sync if the storage does not exists. + // Will return the storage even if it is tombstoned. + auto [dm_storage, is_common_handle, schema_snap] = AtomicGetStorageSchema(new_region, tmt); + if (unlikely(dm_storage == nullptr)) + { + // The storage must be physically dropped, throw exception and do cleanup. + throw Exception("", ErrorCodes::TABLE_IS_DROPPED); + } + + // Get a gc safe point for compacting + Timestamp gc_safepoint = 0; + if (auto pd_client = tmt.getPDClient(); !pd_client->isMock()) + { + gc_safepoint = PDClientHelper::getGCSafePointWithRetry(pd_client, + /* ignore_cache= */ false, + context.getSettingsRef().safe_point_update_interval_seconds); + } + + // Read from SSTs and refine the boundary of blocks output to DTFiles + auto sst_stream = std::make_shared( + new_region, snaps, proxy_helper, dm_storage, schema_snap, gc_safepoint, force_decode, tmt, expected_block_size); + auto bounded_stream + = std::make_shared(sst_stream, ::DB::TiDBPkColumnID, is_common_handle); + stream = std::make_shared(bounded_stream, snapshot_apply_method, job_type, tmt); + + stream->writePrefix(); + stream->write(); + stream->writeSuffix(); + ids = stream->ingestIds(); + break; + } + catch (DB::Exception & e) + { + auto try_clean_up = [&stream]() -> void { + if (stream != nullptr) + stream->cancel(); + }; + if (e.code() == ErrorCodes::REGION_DATA_SCHEMA_UPDATED) + { + // The schema of decoding region data has been updated, need to clear and recreate another stream for writing DTFile(s) + new_region->clearAllData(); + try_clean_up(); + + if (force_decode) + { + // Can not decode data with `force_decode == true`, must be something wrong + throw; + } + + // Update schema and try to decode again + auto metrics = context.getTiFlashMetrics(); + GET_METRIC(metrics, tiflash_schema_trigger_count, type_raft_decode).Increment(); + tmt.getSchemaSyncer()->syncSchemas(context); + // Next time should force_decode + force_decode = true; + + continue; + } + else if (e.code() == ErrorCodes::TABLE_IS_DROPPED) + { + // We can ignore if storage is dropped. + LOG_INFO(log, + "Pre-handle snapshot to DTFiles is ignored because the table is dropped. [region=" << new_region->toString(true) + << "]"); + try_clean_up(); + break; + } + else + { + // Other unrecoverable error, throw + throw; + } + } + } + + return ids; +} + +template +void KVStore::handlePreApplySnapshot(const RegionPtrWrap & new_region, TMTContext & tmt) { LOG_INFO(log, "Try to apply snapshot: " << new_region->toString(true)); @@ -300,6 +421,14 @@ void KVStore::handlePreApplySnapshot(const RegionPtrWithBlock & new_region, TMTC LOG_INFO(log, new_region->toString(false) << " apply snapshot success"); } +template void KVStore::handlePreApplySnapshot(const RegionPtrWithBlock &, TMTContext &); +template void KVStore::handlePreApplySnapshot(const RegionPtrWithSnapshotFiles &, TMTContext &); +template void KVStore::checkAndApplySnapshot(const RegionPtrWithBlock &, TMTContext &); +template void KVStore::checkAndApplySnapshot(const RegionPtrWithSnapshotFiles &, TMTContext &); +template void KVStore::onSnapshot(const RegionPtrWithBlock &, RegionPtr, UInt64, TMTContext &); +template void KVStore::onSnapshot(const RegionPtrWithSnapshotFiles &, RegionPtr, UInt64, TMTContext &); + + static const metapb::Peer & findPeer(const metapb::Region & region, UInt64 peer_id) { for (const auto & peer : region.peers()) @@ -330,10 +459,13 @@ RegionPtr KVStore::genRegionPtr(metapb::Region && region, UInt64 peer_id, UInt64 } void KVStore::handleApplySnapshot( - metapb::Region && region, UInt64 peer_id, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) + metapb::Region && region, uint64_t peer_id, const SSTViewVec snaps, uint64_t index, uint64_t term, TMTContext & tmt) { auto new_region = genRegionPtr(std::move(region), peer_id, index, term); - handlePreApplySnapshot(RegionPtrWithBlock{new_region, preHandleSnapshot(new_region, snaps, tmt)}, tmt); + if (snapshot_apply_method == TiDB::SnapshotApplyMethod::Block) + handlePreApplySnapshot(RegionPtrWithBlock{new_region, preHandleSnapshotToBlock(new_region, snaps, index, term, tmt)}, tmt); + else + handlePreApplySnapshot(RegionPtrWithSnapshotFiles{new_region, preHandleSnapshotToFiles(new_region, snaps, index, term, tmt)}, tmt); } EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) @@ -341,9 +473,10 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec auto region_task_lock = region_manager.genRegionTaskLock(region_id); Stopwatch watch; - auto & ctx = tmt.getContext(); - SCOPE_EXIT( - { GET_METRIC(ctx.getTiFlashMetrics(), tiflash_raft_command_duration_seconds, type_ingest_sst).Observe(watch.elapsedSeconds()); }); + SCOPE_EXIT({ + auto & ctx = tmt.getContext(); + GET_METRIC(ctx.getTiFlashMetrics(), tiflash_raft_command_duration_seconds, type_ingest_sst).Observe(watch.elapsedSeconds()); + }); const RegionPtr region = getRegion(region_id); if (region == nullptr) @@ -354,6 +487,26 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec return EngineStoreApplyRes::NotFound; } + fiu_do_on(FailPoints::force_set_sst_decode_rand, { + static int num_call = 0; + switch (num_call++ % 3) + { + case 0: + snapshot_apply_method = TiDB::SnapshotApplyMethod::Block; + break; + case 1: + snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory; + break; + case 2: + snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Single; + break; + default: + break; + } + LOG_INFO( + log, __FUNCTION__ << ": " << region->toString(true) << " ingest sst by method " << applyMethodToString(snapshot_apply_method)); + }); + const auto func_try_flush = [&]() { if (!region->writeCFCount()) return; @@ -370,10 +523,23 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec } }; - // try to flush remain data in memory. - func_try_flush(); - region->handleIngestSST(snaps, index, term, tmt); - func_try_flush(); + if (snapshot_apply_method == TiDB::SnapshotApplyMethod::Block) + { + // try to flush remain data in memory. + func_try_flush(); + region->handleIngestSSTInMemory(snaps, index, term, tmt); + // after `handleIngestSSTInMemory`, all data are stored in `region`, try to flush committed data into storage + func_try_flush(); + } + else + { + // try to flush remain data in memory. + func_try_flush(); + auto tmp_region = handleIngestSSTByDTFile(region, snaps, index, term, tmt); + region->finishIngestSSTByDTFile(std::move(tmp_region), index, term); + // after `finishIngestSSTByDTFile`, try to flush committed data into storage + func_try_flush(); + } if (region->dataSize()) { @@ -387,4 +553,56 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec } } +RegionPtr KVStore::handleIngestSSTByDTFile(const RegionPtr & region, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) +{ + if (index <= region->appliedIndex()) + return nullptr; + + // Create a tmp region to store uncommitted data + RegionPtr tmp_region; + { + auto meta_region = region->getMetaRegion(); + auto meta_snap = region->dumpRegionMetaSnapshot(); + auto peer_id = meta_snap.peer.id(); + tmp_region = genRegionPtr(std::move(meta_region), peer_id, index, term); + } + + // Decode the KV pairs in ingesting SST into DTFiles + PageIds ingest_ids = preHandleSSTsToDTFiles(tmp_region, snaps, index, term, DM::FileConvertJobType::IngestSST, tmt); + + // If `ingest_ids` is empty, ingest SST won't write delete_range for ingest region, it is safe to + // ignore the step of calling `ingestFiles` + if (!ingest_ids.empty()) + { + auto table_id = region->getMappedTableID(); + if (auto storage = tmt.getStorages().get(table_id); storage) + { + // Ingest DTFiles into DeltaMerge storage + auto & context = tmt.getContext(); + try + { + // acquire lock so that no other threads can drop storage + auto table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + auto dm_storage = std::dynamic_pointer_cast(storage); + auto key_range = DM::RowKeyRange::fromRegionRange( + region->getRange(), table_id, storage->isCommonHandle(), storage->getRowKeyColumnSize()); + // Call `ingestFiles` to ingest external DTFiles. + // Note that ingest sst won't remove the data in the key range + dm_storage->ingestFiles(key_range, ingest_ids, /*clear_data_in_range=*/false, context.getSettingsRef()); + } + catch (DB::Exception & e) + { + // We can ignore if storage is dropped. + if (e.code() == ErrorCodes::TABLE_IS_DROPPED) + return nullptr; + else + throw; + } + } + } + + return tmp_region; +} + + } // namespace DB diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 4563629e687..e220a040c52 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -19,8 +19,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -KVStore::KVStore(Context & context) - : region_persister(context, region_manager), raft_cmd_res(std::make_unique()), log(&Logger::get("KVStore")) +KVStore::KVStore(Context & context, TiDB::SnapshotApplyMethod snapshot_apply_method_) + : region_persister(context, region_manager), + raft_cmd_res(std::make_unique()), + snapshot_apply_method(snapshot_apply_method_), + log(&Logger::get("KVStore")) {} void KVStore::restore(const TiFlashRaftProxyHelper * proxy_helper) diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index fb27e438914..04e09989d54 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -11,6 +12,10 @@ namespace RegionBench { extern void concurrentBatchInsert(const TiDB::TableInfo &, Int64, Int64, Int64, UInt64, UInt64, Context &); } +namespace DM +{ +enum class FileConvertJobType; +} // TODO move to Settings.h static const Seconds REGION_CACHE_GC_PERIOD(60 * 5); @@ -41,7 +46,6 @@ struct WriteCmdsView; enum class EngineStoreApplyRes : uint32_t; struct TiFlashRaftProxyHelper; -struct RegionPtrWithBlock; struct RegionPreDecodeBlockData; using RegionPreDecodeBlockDataPtr = std::unique_ptr; @@ -49,7 +53,7 @@ using RegionPreDecodeBlockDataPtr = std::unique_ptr; class KVStore final : private boost::noncopyable { public: - KVStore(Context & context); + KVStore(Context & context, TiDB::SnapshotApplyMethod snapshot_apply_method_); void restore(const TiFlashRaftProxyHelper *); RegionPtr getRegion(const RegionID region_id) const; @@ -78,8 +82,12 @@ class KVStore final : private boost::noncopyable EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); - RegionPreDecodeBlockDataPtr preHandleSnapshot(RegionPtr new_region, const SSTViewVec, TMTContext & tmt); - void handlePreApplySnapshot(const RegionPtrWithBlock &, TMTContext & tmt); + RegionPreDecodeBlockDataPtr preHandleSnapshotToBlock( + RegionPtr new_region, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); + std::vector /* */ preHandleSnapshotToFiles( + RegionPtr new_region, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); + template + void handlePreApplySnapshot(const RegionPtrWrap &, TMTContext & tmt); void handleDestroy(UInt64 region_id, TMTContext & tmt); void setRegionCompactLogPeriod(UInt64); @@ -87,6 +95,8 @@ class KVStore final : private boost::noncopyable RegionPtr genRegionPtr(metapb::Region && region, UInt64 peer_id, UInt64 index, UInt64 term); const TiFlashRaftProxyHelper * getProxyHelper() const { return proxy_helper; } + TiDB::SnapshotApplyMethod applyMethod() const { return snapshot_apply_method; } + private: friend class MockTiDB; friend struct MockTiDBTable; @@ -95,11 +105,18 @@ class KVStore final : private boost::noncopyable friend void RegionBench::concurrentBatchInsert(const TiDB::TableInfo &, Int64, Int64, Int64, UInt64, UInt64, Context &); using DBGInvokerPrinter = std::function; friend void dbgFuncRemoveRegion(Context &, const ASTs &, DBGInvokerPrinter); - friend void dbgFuncRegionSnapshotWithData(Context &, const ASTs &, DBGInvokerPrinter); friend void dbgFuncPutRegion(Context &, const ASTs &, DBGInvokerPrinter); - void checkAndApplySnapshot(const RegionPtrWithBlock &, TMTContext & tmt); - void onSnapshot(const RegionPtrWithBlock &, RegionPtr old_region, UInt64 old_region_index, TMTContext & tmt); + + std::vector preHandleSSTsToDTFiles( + RegionPtr new_region, const SSTViewVec, uint64_t index, uint64_t term, DM::FileConvertJobType, TMTContext & tmt); + + template + void checkAndApplySnapshot(const RegionPtrWrap &, TMTContext & tmt); + template + void onSnapshot(const RegionPtrWrap &, RegionPtr old_region, UInt64 old_region_index, TMTContext & tmt); + + RegionPtr handleIngestSSTByDTFile(const RegionPtr & region, const SSTViewVec, UInt64 index, UInt64 term, TMTContext & tmt); // Remove region from this TiFlash node. // If region is destroy or moved to another node(change peer), @@ -137,6 +154,8 @@ class KVStore final : private boost::noncopyable // raft_cmd_res stores the result of applying raft cmd. It must be protected by task_mutex. std::unique_ptr raft_cmd_res; + TiDB::SnapshotApplyMethod snapshot_apply_method; + Logger * log; std::atomic REGION_COMPACT_LOG_PERIOD; diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 501171d48a9..3b43a72ba5c 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -27,6 +28,7 @@ extern const char pause_before_apply_raft_snapshot[]; namespace ErrorCodes { extern const int LOGICAL_ERROR; +extern const int REGION_DATA_SCHEMA_UPDATED; extern const int ILLFORMAT_RAFT_ROW; } // namespace ErrorCodes @@ -378,12 +380,12 @@ RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegi region_data_lock); } -/// pre-decode region data into block cache and remove +/// Pre-decode region data into block cache and remove committed data from `region` RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr & region, Context & context) { const auto & tmt = context.getTMTContext(); { - Timestamp gc_safe_point = UINT64_MAX; + Timestamp gc_safe_point = 0; if (auto pd_client = tmt.getPDClient(); !pd_client->isMock()) { gc_safe_point @@ -476,4 +478,158 @@ RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr & regio return std::make_unique(std::move(res_block), schema_version, std::move(*data_list_read)); } +std::tuple, bool, DM::ColumnDefinesPtr> // +AtomicGetStorageSchema(const RegionPtr & region, TMTContext & tmt) +{ + bool is_common_handle = false; + std::shared_ptr dm_storage; + DM::ColumnDefinesPtr schema_snap; + + auto table_id = region->getMappedTableID(); + auto context = tmt.getContext(); + auto metrics = context.getTiFlashMetrics(); + const auto atomicGet = [&](bool force_decode) -> bool { + auto storage = tmt.getStorages().get(table_id); + if (storage == nullptr) + { + if (!force_decode) + return false; + if (storage == nullptr) // Table must have just been GC-ed + return true; + } + auto lock = storage->lockStructure(false, __PRETTY_FUNCTION__); + is_common_handle = storage->isCommonHandle(); + if (unlikely(storage->engineType() != ::TiDB::StorageEngine::DT)) + { + throw Exception("Try to get storage schema with unknown storage engine [table_id=" + DB::toString(table_id) + + "] [engine_type=" + DB::toString(static_cast(storage->engineType())) + "]", + ErrorCodes::LOGICAL_ERROR); + } + if (dm_storage = std::dynamic_pointer_cast(storage); dm_storage != nullptr) + { + auto store = dm_storage->getStore(); + schema_snap = store->getStoreColumns(); + } + return true; + }; + + if (!atomicGet(false)) + { + GET_METRIC(metrics, tiflash_schema_trigger_count, type_raft_decode).Increment(); + tmt.getSchemaSyncer()->syncSchemas(context); + + if (!atomicGet(true)) + throw Exception("Get " + region->toString() + " belonging table " + DB::toString(table_id) + " is_command_handle fail", + ErrorCodes::LOGICAL_ERROR); + } + return std::make_tuple(dm_storage, is_common_handle, schema_snap); +} + +static bool needUpdateSchema(const DM::ColumnDefinesPtr & a, const DM::ColumnDefinesPtr & b) +{ + // Note that we consider `a` is not `b` and need to update schema if either of them is `nullptr` + if (unlikely(a == nullptr || b == nullptr)) + return true; + + // If the two schema is not the same, then it need to be updated. + if (a->size() != b->size()) + return true; + for (size_t i = 0; i < a->size(); ++i) + { + const auto & ca = (*a)[i]; + const auto & cb = (*b)[i]; + + bool col_ok = ca.id == cb.id; + // bool name_ok = ca.name == cb.name; + bool type_ok = ca.type->equals(*cb.type); + + if (!col_ok || !type_ok) + return true; + } + return false; +} + +static Block sortColumnsBySchemaSnap(Block && ori, const DM::ColumnDefines & schema) +{ +#ifndef NDEBUG + // Some trival check to ensure the input is legal + if (ori.columns() != schema.size()) + { + throw Exception("Try to sortColumnsBySchemaSnap with different column size [block_columns=" + DB::toString(ori.columns()) + + "] [schema_columns=" + DB::toString(schema.size()) + "]"); + } +#endif + + std::map index_by_cid; + for (size_t i = 0; i < ori.columns(); ++i) + { + const ColumnWithTypeAndName & c = ori.getByPosition(i); + index_by_cid[c.column_id] = i; + } + + Block res; + for (const auto & cd : schema) + { + res.insert(ori.getByPosition(index_by_cid[cd.id])); + } +#ifndef NDEBUG + assertBlocksHaveEqualStructure(res, DM::toEmptyBlock(schema), "sortColumnsBySchemaSnap"); +#endif + + return res; +} + +/// Decode region data into block and belonging schema snapshot, remove committed data from `region` +/// The return value is a block that store the committed data scanned and removed from `region`. +/// The columns of returned block is sorted by `schema_snap`. +Block GenRegionBlockDatawithSchema(const RegionPtr & region, + const std::shared_ptr & dm_storage, + const DM::ColumnDefinesPtr & schema_snap, + Timestamp gc_safepoint, + bool force_decode, + TMTContext & tmt) +{ + // In 5.0.1, feature `compaction filter` is enabled by default. Under such feature tikv will do gc in write & default cf individually. + // If some rows were updated and add tiflash replica, tiflash store may receive region snapshot with unmatched data in write & default cf sst files. + region->tryCompactionFilter(gc_safepoint); + + std::optional data_list_read = std::nullopt; + data_list_read = ReadRegionCommitCache(region); + + Block res_block; + // No committed data, just return + if (!data_list_read) + return std::move(res_block); + + auto context = tmt.getContext(); + auto metrics = context.getTiFlashMetrics(); + + { + Stopwatch watch; + auto lock = dm_storage->lockStructure(false, __PRETTY_FUNCTION__); + // Compare schema_snap with current schema, throw exception if changed. + auto store = dm_storage->getStore(); + auto cur_schema_snap = store->getStoreColumns(); + if (needUpdateSchema(cur_schema_snap, schema_snap)) + throw Exception("", ErrorCodes::REGION_DATA_SCHEMA_UPDATED); + + auto reader = RegionBlockReader(dm_storage); + auto [block, ok] = reader.read(*data_list_read, force_decode); + if (unlikely(!ok)) + throw Exception("", ErrorCodes::REGION_DATA_SCHEMA_UPDATED); + + GET_METRIC(metrics, tiflash_raft_write_data_to_storage_duration_seconds, type_decode).Observe(watch.elapsedSeconds()); + + // For DeltaMergeStore, we always store an extra column with column_id = -1 + res_block = store->addExtraColumnIfNeed(context, std::move(block)); + } + + res_block = sortColumnsBySchemaSnap(std::move(res_block), *schema_snap); + + // Remove committed data + RemoveRegionCommitCache(region, *data_list_read); + + return std::move(res_block); +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 8c843a85db0..d1cabf568da 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -212,10 +212,11 @@ BatchReadIndexRes TiFlashRaftProxyHelper::batchReadIndex(const std::vector && ids_) : region(region_), ingest_ids(std::move(ids_)) + { + CurrentMetrics::add(CurrentMetrics::RaftNumSnapshotsPendingApply); + } + RegionPtr region; + std::vector ingest_ids; // The file_ids storing pre-handled files +}; + RawCppPtr PreHandleSnapshot( EngineStoreServerWrap * server, BaseBuffView region_buff, uint64_t peer_id, SSTViewVec snaps, uint64_t index, uint64_t term) { @@ -233,9 +245,26 @@ RawCppPtr PreHandleSnapshot( auto & tmt = *server->tmt; auto & kvstore = tmt.getKVStore(); auto new_region = kvstore->genRegionPtr(std::move(region), peer_id, index, term); - auto new_region_block_cache = kvstore->preHandleSnapshot(new_region, snaps, tmt); - auto res = new PreHandledSnapshot{new_region, std::move(new_region_block_cache)}; - return GenRawCppPtr(res, RawCppPtrTypeImpl::PreHandledSnapshot); + switch (kvstore->applyMethod()) + { + case TiDB::SnapshotApplyMethod::Block: + { + // Pre-decode as a block + auto new_region_block_cache = kvstore->preHandleSnapshotToBlock(new_region, snaps, index, term, tmt); + auto res = new PreHandledSnapshotWithBlock{new_region, std::move(new_region_block_cache)}; + return GenRawCppPtr(res, RawCppPtrTypeImpl::PreHandledSnapshotWithBlock); + } + case TiDB::SnapshotApplyMethod::DTFile_Directory: + case TiDB::SnapshotApplyMethod::DTFile_Single: + { + // Pre-decode and save as DTFiles + auto ingest_ids = kvstore->preHandleSnapshotToFiles(new_region, snaps, index, term, tmt); + auto res = new PreHandledSnapshotWithFiles{new_region, std::move(ingest_ids)}; + return GenRawCppPtr(res, RawCppPtrTypeImpl::PreHandledSnapshotWithFiles); + } + default: + throw Exception("Unknow Region apply method: " + applyMethodToString(kvstore->applyMethod())); + } } catch (...) { @@ -244,12 +273,24 @@ RawCppPtr PreHandleSnapshot( } } +template void ApplyPreHandledSnapshot(EngineStoreServerWrap * server, PreHandledSnapshot * snap) { + static_assert( + std::is_same_v || std::is_same_v, + "Unknown pre-handled snapshot type"); + try { auto & kvstore = server->tmt->getKVStore(); - kvstore->handlePreApplySnapshot(RegionPtrWithBlock{snap->region, std::move(snap->cache)}, *server->tmt); + if constexpr (std::is_same_v) + { + kvstore->handlePreApplySnapshot(RegionPtrWithBlock{snap->region, std::move(snap->cache)}, *server->tmt); + } + else if constexpr (std::is_same_v) + { + kvstore->handlePreApplySnapshot(RegionPtrWithSnapshotFiles{snap->region, std::move(snap->ingest_ids)}, *server->tmt); + } } catch (...) { @@ -262,9 +303,15 @@ void ApplyPreHandledSnapshot(EngineStoreServerWrap * server, RawVoidPtr res, Raw { switch (static_cast(type)) { - case RawCppPtrTypeImpl::PreHandledSnapshot: + case RawCppPtrTypeImpl::PreHandledSnapshotWithBlock: { - PreHandledSnapshot * snap = reinterpret_cast(res); + auto * snap = reinterpret_cast(res); + ApplyPreHandledSnapshot(server, snap); + break; + } + case RawCppPtrTypeImpl::PreHandledSnapshotWithFiles: + { + auto * snap = reinterpret_cast(res); ApplyPreHandledSnapshot(server, snap); break; } @@ -283,8 +330,11 @@ void GcRawCppPtr(EngineStoreServerWrap *, RawVoidPtr ptr, RawCppPtrType type) case RawCppPtrTypeImpl::String: delete reinterpret_cast(ptr); break; - case RawCppPtrTypeImpl::PreHandledSnapshot: - delete reinterpret_cast(ptr); + case RawCppPtrTypeImpl::PreHandledSnapshotWithBlock: + delete reinterpret_cast(ptr); + break; + case RawCppPtrTypeImpl::PreHandledSnapshotWithFiles: + delete reinterpret_cast(ptr); break; default: LOG_ERROR(&Logger::get(__PRETTY_FUNCTION__), "unknown type " + std::to_string(uint32_t(type))); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index 05e9cf19f09..78a51da1eb9 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -37,7 +37,8 @@ enum class RawCppPtrTypeImpl : RawCppPtrType { None = 0, String, - PreHandledSnapshot, + PreHandledSnapshotWithBlock, + PreHandledSnapshotWithFiles, }; RawCppPtr GenRawCppPtr(RawVoidPtr ptr_ = nullptr, RawCppPtrTypeImpl type_ = RawCppPtrTypeImpl::None); diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index 3e35dc825ef..f6f78d432e4 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -24,8 +24,6 @@ extern const int UNKNOWN_FORMAT_VERSION; const UInt32 Region::CURRENT_VERSION = 1; -const std::string Region::log_name = "Region"; - RegionData::WriteCFIter Region::removeDataByWriteIt(const RegionData::WriteCFIter & write_it) { return data.removeDataByWriteIt(write_it); } RegionDataReadInfo Region::readDataByWriteIt(const RegionData::ConstWriteCFIter & write_it, bool need_value) const @@ -66,6 +64,12 @@ void Region::remove(const std::string & cf, const TiKVKey & key) void Region::doRemove(ColumnFamilyType type, const TiKVKey & key) { data.remove(type, key); } +void Region::clearAllData() +{ + std::unique_lock lock(mutex); + data = RegionData(); +} + UInt64 Region::appliedIndex() const { return meta.appliedIndex(); } RegionPtr Region::splitInto(RegionMeta && meta) @@ -728,7 +732,7 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 return EngineStoreApplyRes::None; } -void Region::handleIngestSST(const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) +void Region::handleIngestSSTInMemory(const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) { if (index <= appliedIndex()) return; @@ -765,6 +769,29 @@ void Region::handleIngestSST(const SSTViewVec snaps, UInt64 index, UInt64 term, meta.notifyAll(); } +void Region::finishIngestSSTByDTFile(RegionPtr && rhs, UInt64 index, UInt64 term) +{ + if (index <= appliedIndex()) + return; + + { + std::unique_lock lock(mutex); + + if (rhs) + { + // Merge the uncommitted data from `rhs` + // (we have taken the ownership of `rhs`, so don't acquire lock on `rhs.mutex`) + data.mergeFrom(rhs->data); + } + + meta.setApplied(index, term); + } + LOG_INFO(log, + __FUNCTION__ << ": " << this->toString(false) << " finish to ingest sst by DTFile [write_cf_keys=" << data.write_cf.getSize() + << "] [default_cf_keys=" << data.default_cf.getSize() << "] [lock_cf_keys=" << data.lock_cf.getSize() << "]"); + meta.notifyAll(); +} + RegionRaftCommandDelegate & Region::makeRaftCommandDelegate(const KVStoreTaskLock & lock) { static_assert(sizeof(RegionRaftCommandDelegate) == sizeof(Region)); @@ -778,7 +805,7 @@ RegionMetaSnapshot Region::dumpRegionMetaSnapshot() const { return meta.dumpRegi Region::Region(RegionMeta && meta_) : Region(std::move(meta_), nullptr) {} Region::Region(DB::RegionMeta && meta_, const TiFlashRaftProxyHelper * proxy_helper_) - : meta(std::move(meta_)), log(&Logger::get(log_name)), mapped_table_id(meta.getRange()->getMappedTableID()), proxy_helper(proxy_helper_) + : meta(std::move(meta_)), log(&Logger::get("Region")), mapped_table_id(meta.getRange()->getMappedTableID()), proxy_helper(proxy_helper_) {} TableID Region::getMappedTableID() const { return mapped_table_id; } diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index a2da943a293..d016c8d03ce 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -40,8 +40,6 @@ class Region : public std::enable_shared_from_this public: const static UInt32 CURRENT_VERSION; - const static std::string log_name; - static const auto PutFlag = RecordKVFormat::CFModifyFlag::PutFlag; static const auto DelFlag = RecordKVFormat::CFModifyFlag::DelFlag; @@ -106,6 +104,9 @@ class Region : public std::enable_shared_from_this void insert(ColumnFamilyType cf, TiKVKey && key, TiKVValue && value); void remove(const std::string & cf, const TiKVKey & key); + // Directly drop all data in this Region object. + void clearAllData(); + CommittedScanner createCommittedScanner(bool use_lock = true); CommittedRemover createCommittedRemover(bool use_lock = true); @@ -170,7 +171,8 @@ class Region : public std::enable_shared_from_this TableID getMappedTableID() const; EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt); - void handleIngestSST(const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt); + void handleIngestSSTInMemory(const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt); + void finishIngestSSTByDTFile(RegionPtr && rhs, UInt64 index, UInt64 term); UInt64 getSnapshotEventFlag() const { return snapshot_event_flag; } diff --git a/dbms/src/Storages/Transaction/RegionData.cpp b/dbms/src/Storages/Transaction/RegionData.cpp index 014c5cce484..78eb5080586 100644 --- a/dbms/src/Storages/Transaction/RegionData.cpp +++ b/dbms/src/Storages/Transaction/RegionData.cpp @@ -216,9 +216,22 @@ bool RegionData::isEqual(const RegionData & r2) const } RegionData::RegionData(RegionData && data) - : write_cf(std::move(data.write_cf)), default_cf(std::move(data.default_cf)), lock_cf(std::move(data.lock_cf)) + : write_cf(std::move(data.write_cf)), + default_cf(std::move(data.default_cf)), + lock_cf(std::move(data.lock_cf)), + cf_data_size(data.cf_data_size.load()) {} +RegionData & RegionData::operator=(RegionData && rhs) +{ + write_cf = std::move(rhs.write_cf); + default_cf = std::move(rhs.default_cf); + lock_cf = std::move(rhs.lock_cf); + cf_data_size = rhs.cf_data_size.load(); + return *this; +} + + UInt8 RegionData::getWriteType(const ConstWriteCFIter & write_it) { return RegionWriteCFDataTrait::getWriteType(write_it->second); } const RegionDefaultCFDataTrait::Map & RegionData::getDefaultCFMap(RegionWriteCFData * write) diff --git a/dbms/src/Storages/Transaction/RegionData.h b/dbms/src/Storages/Transaction/RegionData.h index 0ef20c5f068..ff901e8ab75 100644 --- a/dbms/src/Storages/Transaction/RegionData.h +++ b/dbms/src/Storages/Transaction/RegionData.h @@ -58,6 +58,7 @@ class RegionData RegionData() {} RegionData(RegionData && data); + RegionData& operator=(RegionData && data); public: static UInt8 getWriteType(const ConstWriteCFIter & write_it); diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 6d42fed0b00..d55de57f9dc 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -36,6 +36,7 @@ struct MockTiDBTable; class RegionRangeKeys; class RegionTaskLock; struct RegionPtrWithBlock; +struct RegionPtrWithSnapshotFiles; class RegionScanFilter; using RegionScanFilterPtr = std::shared_ptr; @@ -204,6 +205,7 @@ class RegionTable : private boost::noncopyable Logger * log; }; + // Block cache of region data with schema version. struct RegionPreDecodeBlockData { @@ -245,4 +247,24 @@ struct RegionPtrWithBlock CachePtr pre_decode_cache; }; + +// A wrap of RegionPtr, with snapshot files directory waitting to be ingested +struct RegionPtrWithSnapshotFiles +{ + using Base = RegionPtr; + + /// can accept const ref of RegionPtr without cache + RegionPtrWithSnapshotFiles(const Base & base_, std::vector ids_ = {}) : base(base_), ingest_ids(std::move(ids_)) {} + + /// to be compatible with usage as RegionPtr. + Base::element_type * operator->() const { return base.operator->(); } + const Base::element_type & operator*() const { return base.operator*(); } + + /// make it could be cast into RegionPtr implicitly. + operator const Base &() const { return base; } + + const Base & base; + const std::vector ingest_ids; +}; + } // namespace DB diff --git a/dbms/src/Storages/Transaction/RowCodec.cpp b/dbms/src/Storages/Transaction/RowCodec.cpp index dceb539e3e7..19c2f77f64b 100644 --- a/dbms/src/Storages/Transaction/RowCodec.cpp +++ b/dbms/src/Storages/Transaction/RowCodec.cpp @@ -393,18 +393,20 @@ void encodeRowV1(const TiDB::TableInfo & table_info, const std::vector & column_in_key = 1; else if (table_info.is_common_handle) column_in_key = table_info.getPrimaryIndexInfo().idx_cols.size(); - if (table_info.columns.size() != fields.size() + column_in_key) + if (table_info.columns.size() < fields.size() + column_in_key) throw Exception(std::string("Encoding row has ") + std::to_string(table_info.columns.size()) + " columns but " + std::to_string(fields.size() + table_info.pk_is_handle) + " values: ", ErrorCodes::LOGICAL_ERROR); - size_t index = 0; + size_t encoded_fields_idx = 0; for (auto & column_info : table_info.columns) { if ((table_info.pk_is_handle || table_info.is_common_handle) && column_info.hasPriKeyFlag()) continue; EncodeDatum(Field(column_info.id), TiDB::CodecFlagInt, ss); - EncodeDatumForRow(fields[index++], column_info.getCodecFlag(), ss, column_info); + EncodeDatumForRow(fields[encoded_fields_idx++], column_info.getCodecFlag(), ss, column_info); + if (encoded_fields_idx == fields.size()) + break; } } @@ -419,7 +421,7 @@ struct RowEncoderV2 column_in_key = 1; else if (table_info.is_common_handle) column_in_key = table_info.getPrimaryIndexInfo().idx_cols.size(); - if (table_info.columns.size() != fields.size() + column_in_key) + if (table_info.columns.size() < fields.size() + column_in_key) throw Exception(std::string("Encoding row has ") + std::to_string(table_info.columns.size()) + " columns but " + std::to_string(fields.size() + table_info.pk_is_handle) + " values: ", ErrorCodes::LOGICAL_ERROR); @@ -449,6 +451,9 @@ struct RowEncoderV2 null_column_ids.emplace(column_info.id); } i_val++; + + if (i_val == fields.size()) + break; } is_big = is_big || value_length > std::numeric_limits::ValueOffsetType>::max(); diff --git a/dbms/src/Storages/Transaction/StorageEngineType.h b/dbms/src/Storages/Transaction/StorageEngineType.h index 23c68c33749..938804bfc10 100644 --- a/dbms/src/Storages/Transaction/StorageEngineType.h +++ b/dbms/src/Storages/Transaction/StorageEngineType.h @@ -1,5 +1,8 @@ #pragma once +#include +#include + namespace TiDB { @@ -14,4 +17,28 @@ enum class StorageEngine UNSUPPORTED_ENGINES = 128, }; +enum class SnapshotApplyMethod : std::int32_t +{ + Block = 1, + // Invalid if the storage engine is not DeltaTree + DTFile_Directory, + DTFile_Single, +}; + +inline const std::string applyMethodToString(SnapshotApplyMethod method) +{ + switch (method) + { + case SnapshotApplyMethod::Block: + return "block"; + case SnapshotApplyMethod::DTFile_Directory: + return "file1"; + case SnapshotApplyMethod::DTFile_Single: + return "file2"; + default: + return "unknown(" + std::to_string(static_cast(method)) + ")"; + } + return "unknown"; +} + } // namespace TiDB diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index c5d7877a5ab..44232eeb2be 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -16,7 +16,7 @@ namespace DB TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config) : context(context_), - kvstore(std::make_shared(context)), + kvstore(std::make_shared(context, raft_config.snapshot_apply_method)), region_table(context), background_service(nullptr), gc_manager(context), diff --git a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp index 75a50a81c72..a19ffb82e73 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include diff --git a/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp b/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp index 65969c20e68..aa95fc24ad6 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_type_mapping.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include namespace DB diff --git a/dbms/src/TestUtils/TiFlashTestBasic.h b/dbms/src/TestUtils/TiFlashTestBasic.h index 346cb4b53fd..8c25f458554 100644 --- a/dbms/src/TestUtils/TiFlashTestBasic.h +++ b/dbms/src/TestUtils/TiFlashTestBasic.h @@ -10,8 +10,24 @@ #include #include #include + +#if !__clang__ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" +#else +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wsign-compare" +#endif + #include +#if !__clang__ +#pragma GCC diagnostic pop +#else +#pragma clang diagnostic pop +#endif + + namespace DB { namespace tests diff --git a/dbms/src/TestUtils/gtests_dbms_main.cpp b/dbms/src/TestUtils/gtests_dbms_main.cpp index fd929323d2c..f3866ba352d 100644 --- a/dbms/src/TestUtils/gtests_dbms_main.cpp +++ b/dbms/src/TestUtils/gtests_dbms_main.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB::tests { diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 219576e824c..59d7ff7c008 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1595916828338, + "iteration": 1619587165566, "links": [], "panels": [ { @@ -2251,13 +2251,18 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/delete_range|ingest/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_command_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(increase(tiflash_storage_command_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2300,11 +2305,11 @@ "show": true }, { - "format": "none", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -2348,7 +2353,16 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/5min-write/", + "yaxis": 2 + }, + { + "alias": "/5min-all/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, @@ -2384,6 +2398,22 @@ "intervalFactor": 1, "legendFormat": "30min-{{instance}}", "refId": "D" + }, + { + "refId": "E", + "expr": "sum((rate(tiflash_system_profile_event_PSMWriteBytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) + rate(tiflash_system_profile_event_WriteBufferFromFileDescriptorWriteBytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]) + rate(tiflash_system_profile_event_WriteBufferAIOWriteBytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m]))) by (instance)", + "hide": true, + "intervalFactor": 1, + "format": "time_series", + "legendFormat": "5min-all-{{instance}}" + }, + { + "refId": "F", + "expr": "sum(rate(tiflash_system_profile_event_DMWriteBytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[5m])) by (instance)", + "hide": true, + "intervalFactor": 1, + "format": "time_series", + "legendFormat": "5min-write-{{instance}}" } ], "thresholds": [], @@ -2415,7 +2445,7 @@ "show": true }, { - "format": "none", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -2689,17 +2719,31 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/(delta_merge)|(seg_)/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_subtask_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tiflash_storage_subtask_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"delta_merge|delta_merge_fg|delta_merge_bg_gc|seg_merge|seg_split\"}[1m])) by (type)", "format": "time_series", + "hide": false, "intervalFactor": 1, "legendFormat": "{{type}}", "refId": "A" + }, + { + "expr": "sum(increase(tiflash_storage_subtask_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"delta_merge|delta_merge_fg|delta_merge_bg_gc|seg_merge|seg_split\"}[1m])) by (type)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "B" } ], "thresholds": [], @@ -2731,11 +2775,11 @@ "show": true }, { - "format": "none", + "format": "opm", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -2780,7 +2824,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "99-delta_merge", + "alias": "/^.*-delta_merge/", "yaxis": 2 } ], @@ -2961,6 +3005,7 @@ "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", @@ -3760,7 +3805,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"}[1m]))", + "expr": "sum(rate(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m]))", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -3769,21 +3814,21 @@ "step": 10 }, { - "expr": "sum(rate(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!=\"write\"}[1m]))", + "expr": "sum(rate(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"}[1m]))", "format": "time_series", "intervalFactor": 1, "legendFormat": "throughput_delta-management", "refId": "B" }, { - "expr": "sum(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"write\"})", + "expr": "sum(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"})", "format": "time_series", "intervalFactor": 1, "legendFormat": "total_write", "refId": "C" }, { - "expr": "sum(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!=\"write\"})", + "expr": "sum(tiflash_storage_throughput_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"})", "format": "time_series", "intervalFactor": 1, "legendFormat": "total_delta-management", @@ -4544,6 +4589,7 @@ "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", @@ -4609,6 +4655,7 @@ "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", @@ -4674,6 +4721,7 @@ "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", @@ -4902,6 +4950,7 @@ "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", @@ -4967,6 +5016,7 @@ "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", diff --git a/tests/delta-merge-test/raft/ingest_sst.test b/tests/delta-merge-test/raft/ingest_sst.test index 27d3c30d1a4..ac512888e30 100644 --- a/tests/delta-merge-test/raft/ingest_sst.test +++ b/tests/delta-merge-test/raft/ingest_sst.test @@ -1,4 +1,5 @@ => DBGInvoke __enable_schema_sync_service('true') +=> DBGInvoke __clean_up_region() => DBGInvoke __drop_tidb_table(default, test) => drop table if exists default.test @@ -18,8 +19,6 @@ │ -4 │ 4 │ │ -5 │ 5 │ │ -7 │ 7 │ -└───────┴─────────────┘ -┌─col_1─┬─_tidb_rowid─┐ │ -8 │ 8 │ └───────┴─────────────┘ => DBGInvoke __drop_tidb_table(default, test) diff --git a/tests/delta-merge-test/raft/schema/drop_on_restart.test b/tests/delta-merge-test/raft/schema/drop_on_restart.test index cc88166bd25..4ae8518e7e5 100644 --- a/tests/delta-merge-test/raft/schema/drop_on_restart.test +++ b/tests/delta-merge-test/raft/schema/drop_on_restart.test @@ -58,3 +58,10 @@ │ default │ │ system │ └─────────┘ + +# Physical cleanup so that it won't make trouble for other tests +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> drop database if exists db3 +=> drop database if exists db4 +=> drop database if exists db5 diff --git a/tests/delta-merge-test/raft/snapshot_dtfile.test b/tests/delta-merge-test/raft/snapshot_dtfile.test new file mode 100644 index 00000000000..5982b554030 --- /dev/null +++ b/tests/delta-merge-test/raft/snapshot_dtfile.test @@ -0,0 +1,168 @@ +# Disable background schema sync to test schema sync triggled by applying snapshot +=> DBGInvoke __enable_schema_sync_service('false') +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __clean_up_region() + +##### +## Pre-handle region to dt files then apply +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __region_snapshot(4, 0, 10000, default, test) +=> DBGInvoke __refresh_schemas() +=> DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 6, 'col_1 Int64', '') +┌─region_snapshot_pre_handle_file(default, test, 4, 3, 6)────────┐ +│ Generate 1 files for [region_id=4] │ +└────────────────────────────────────────────────────────────────┘ +=> DBGInvoke __region_snapshot_apply_file(4) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ -3 │ 3 │ +│ -4 │ 4 │ +│ -5 │ 5 │ +└───────┴─────────────┘ + +##### +## Mock to test idempotence of applying snapshot +=> DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 6, 'col_1 Int64', '') +┌─region_snapshot_pre_handle_file(default, test, 4, 3, 6)────────┐ +│ Generate 1 files for [region_id=4] │ +└────────────────────────────────────────────────────────────────┘ +## Mock that we crashed before applying first snapshot, then replay the process of apply snapshot +=> DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 6, 'col_1 Int64', '') +┌─region_snapshot_pre_handle_file(default, test, 4, 3, 6)────────┐ +│ Generate 1 files for [region_id=4] │ +└────────────────────────────────────────────────────────────────┘ +=> DBGInvoke __region_snapshot_apply_file(4) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ -3 │ 3 │ +│ -4 │ 4 │ +│ -5 │ 5 │ +└───────┴─────────────┘ + +##### +### Mock to test apply snapshot with multiple schema +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 String') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 UInt64') +=> DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 12, 'col_1 Int64, col_2 String, col_3 UInt64', '', 3) +┌─region_snapshot_pre_handle_file(default, test, 4, 3, 12, "col_1 Int64, col_2 String, col_3 UInt64", "", 3)─┐ +│ Generate 1 files for [region_id=4] │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +=> DBGInvoke region_snapshot_apply_file(4) +┌─__region_snapshot_apply_file(4)──────┐ +│ success apply region 4 with dt files │ +└──────────────────────────────────────┘ +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┬─col_2─┬─col_3─┐ +│ -3 │ 3 │ │ 0 │ +│ -4 │ 4 │ │ 0 │ +│ -5 │ 5 │ │ 0 │ +│ -6 │ 6 │ _6 │ 0 │ +│ -7 │ 7 │ _7 │ 0 │ +│ -8 │ 8 │ _8 │ 0 │ +│ -9 │ 9 │ _9 │ 4 │ +│ -10 │ 10 │ _10 │ 5 │ +│ -11 │ 11 │ _11 │ 5 │ +└───────┴─────────────┴───────┴───────┘ + +##### +## Recreate test table +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __region_snapshot(4, 0, 10000, default, test) +=> DBGInvoke __refresh_schemas() +## Mock to test apply snapshot with an older schema (case 1 - add column) +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 String') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 UInt64') +=> DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 12, 'col_1 Int64, col_2 String, col_3 UInt64', '', 3) +┌─region_snapshot_pre_handle_file(default, test, 4, 3, 12, "col_1 Int64, col_2 String, col_3 UInt64", "", 3)─┐ +│ Generate 1 files for [region_id=4] │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +## Add column so that we should fill defaut value for new-added columns +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_4 Nullable(UInt64)') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_5 Nullable(String)') +=> DBGInvoke __refresh_schemas() +=> DBGInvoke region_snapshot_apply_file(4) +┌─region_snapshot_apply_file(4)────────┐ +│ success apply region 4 with dt files │ +└──────────────────────────────────────┘ +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┬─col_2─┬─col_3─┬─col_4─┬─col_5─┐ +│ -3 │ 3 │ │ 0 │ \N │ \N │ +│ -4 │ 4 │ │ 0 │ \N │ \N │ +│ -5 │ 5 │ │ 0 │ \N │ \N │ +│ -6 │ 6 │ _6 │ 0 │ \N │ \N │ +│ -7 │ 7 │ _7 │ 0 │ \N │ \N │ +│ -8 │ 8 │ _8 │ 0 │ \N │ \N │ +│ -9 │ 9 │ _9 │ 4 │ \N │ \N │ +│ -10 │ 10 │ _10 │ 5 │ \N │ \N │ +│ -11 │ 11 │ _11 │ 5 │ \N │ \N │ +└───────┴─────────────┴───────┴───────┴───────┴───────┘ + +##### +## Recreate test table +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __region_snapshot(4, 0, 10000, default, test) +=> DBGInvoke __refresh_schemas() +## Mock to test apply snapshot with an older schema (case 2 - drop column) +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 String') +=> DBGInvoke __add_column_to_tidb_table(default, test, 'col_3 UInt64') +=> DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 12, 'col_1 Int64, col_2 String, col_3 UInt64', '', 3) +┌─region_snapshot_pre_handle_file(default, test, 4, 3, 12, "col_1 Int64, col_2 String, col_3 UInt64", "", 3)─┐ +│ Generate 1 files for [region_id=4] │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +## Drop column so that the Raft data always contain more column for decoding +=> DBGInvoke __drop_column_from_tidb_table(default, test, col_2) +=> DBGInvoke __refresh_schemas() +=> DBGInvoke region_snapshot_apply_file(4) +┌─region_snapshot_apply_file(4)────────┐ +│ success apply region 4 with dt files │ +└──────────────────────────────────────┘ +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┬─col_3─┐ +│ -3 │ 3 │ 0 │ +│ -4 │ 4 │ 0 │ +│ -5 │ 5 │ 0 │ +│ -6 │ 6 │ 0 │ +│ -7 │ 7 │ 0 │ +│ -8 │ 8 │ 0 │ +│ -9 │ 9 │ 4 │ +│ -10 │ 10 │ 5 │ +│ -11 │ 11 │ 5 │ +└───────┴─────────────┴───────┘ + +##### +## Test that BR/lightning may only ingest sst files of write cf into tikv without default cf +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __region_snapshot(4, 0, 10000, default, test) +=> DBGInvoke __refresh_schemas() +# Only the sst files of write cf ingested and send to TiFlash as a snapshot, no panic, and those uncommitted data remain in memory +=> DBGInvoke __region_snapshot_pre_handle_file(default, test, 4, 3, 12, 'col_1 Int64', '', 1, 'write') +=> DBGInvoke __region_snapshot_apply_file(4) +# There should be no committed rows ingested +=> select * from default.test + +# Apply a snapshot with write & default cfs +=> DBGInvoke __region_snapshot_pre_handle_file(default, test, 4, 3, 12, 'col_1 Int64', '', 1, 'write,default') +=> DBGInvoke __region_snapshot_apply_file(4) +=> select * from default.test +┌─col_1─┬─_tidb_rowid─┐ +│ -3 │ 3 │ +│ -4 │ 4 │ +│ -5 │ 5 │ +│ -6 │ 6 │ +│ -7 │ 7 │ +│ -8 │ 8 │ +│ -9 │ 9 │ +│ -10 │ 10 │ +│ -11 │ 11 │ +└───────┴─────────────┘ + +=> DBGInvoke __drop_tidb_table(default, test) +=> drop table if exists default.test +=> DBGInvoke __enable_schema_sync_service('true') diff --git a/tests/docker/config/tics_tmt.toml b/tests/docker/config/tics_tmt.toml index ed8a1e6ec8d..08ec3806595 100644 --- a/tests/docker/config/tics_tmt.toml +++ b/tests/docker/config/tics_tmt.toml @@ -21,3 +21,5 @@ http_port = 8123 [raft] # specify which storage engine we use. tmt or dt storage_engine = "tmt" +[raft.snapshot] + method = "block" diff --git a/tests/docker/config/tiflash_dt.toml b/tests/docker/config/tiflash_dt.toml index 30c385a00e5..310460a0b33 100644 --- a/tests/docker/config/tiflash_dt.toml +++ b/tests/docker/config/tiflash_dt.toml @@ -92,3 +92,8 @@ http_port = 8123 # Deprecated Raft data storage path setting style. Check [storage.raft] section for new style. # If it is not set, it will be the first path of "path" appended with "/kvstore". # kvstore_path = "" + + [raft.snapshot] + # The way to apply snapshot data + # The value is one of "block" / "file1" / "file2". + method = "file1"