From 3e20625cc6558a3758e6b07cc8b364dc10c03a0f Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Wed, 26 Jun 2024 10:23:59 +0800 Subject: [PATCH] storage: Support adding vector index in background (#203) Signed-off-by: Wish Signed-off-by: Lloyd-Pottiger Co-authored-by: Wish Co-authored-by: JaySon-Huang --- dbms/src/Common/CurrentMetrics.cpp | 1 + dbms/src/Interpreters/Context.cpp | 23 + dbms/src/Interpreters/Context.h | 5 + dbms/src/Server/Server.cpp | 29 +- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 109 +++- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 149 ++++- .../DeltaMergeStore_InternalSegment.cpp | 417 ++++++++++++- .../DeltaMerge/DeltaMergeStore_Statistics.cpp | 54 ++ dbms/src/Storages/DeltaMerge/File/DMFile.cpp | 14 +- dbms/src/Storages/DeltaMerge/File/DMFile.h | 14 +- .../DeltaMerge/File/DMFileIndexWriter.cpp | 229 +++++++ .../DeltaMerge/File/DMFileIndexWriter.h | 84 +++ .../File/DMFileV3IncrementWriter.cpp | 13 +- .../DeltaMerge/File/DMFileV3IncrementWriter.h | 4 +- .../Storages/DeltaMerge/File/DMFileWriter.cpp | 41 +- .../Storages/DeltaMerge/File/DMFileWriter.h | 7 +- .../src/Storages/DeltaMerge/Index/IndexInfo.h | 41 ++ .../DeltaMerge/LocalIndexerScheduler.cpp | 422 +++++++++++++ .../DeltaMerge/LocalIndexerScheduler.h | 208 +++++++ .../DeltaMerge/LocalIndexerScheduler_fwd.h | 26 + dbms/src/Storages/DeltaMerge/Segment.cpp | 91 ++- dbms/src/Storages/DeltaMerge/Segment.h | 1 - .../Storages/DeltaMerge/StableValueSpace.cpp | 7 +- .../tests/gtest_dm_delta_merge_store.cpp | 8 +- ...est_dm_delta_merge_store_fast_add_peer.cpp | 3 +- .../gtest_dm_delta_merge_store_test_basic.h | 6 +- ...test_dm_delta_merge_store_vector_index.cpp | 569 +++++++++++++++++ .../tests/gtest_dm_minmax_index.cpp | 5 +- .../tests/gtest_dm_simple_pk_test_basic.cpp | 3 +- .../tests/gtest_dm_vector_index.cpp | 147 +++-- .../tests/gtest_dm_vector_index_utils.h | 88 +++ .../tests/gtest_local_indexer_scheduler.cpp | 589 ++++++++++++++++++ .../tests/gtest_segment_test_basic.cpp | 36 ++ .../tests/gtest_segment_test_basic.h | 5 + .../DeltaMerge/tests/gtest_segment_util.h | 2 + .../DeltaMerge/workload/DTWorkload.cpp | 3 +- dbms/src/Storages/FormatVersion.h | 2 +- dbms/src/Storages/StorageDeltaMerge.cpp | 40 +- .../System/StorageSystemDTLocalIndexes.cpp | 138 ++++ .../System/StorageSystemDTLocalIndexes.h | 50 ++ .../System/StorageSystemDTSegments.cpp | 2 + .../Storages/System/StorageSystemDTTables.cpp | 2 + .../Storages/System/attachSystemTables.cpp | 2 + dbms/src/TestUtils/TiFlashTestEnv.cpp | 2 + dbms/src/TiDB/Schema/TiDB.cpp | 8 + dbms/src/TiDB/Schema/TiDB.h | 2 + 46 files changed, 3464 insertions(+), 237 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp create mode 100644 dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.h create mode 100644 dbms/src/Storages/DeltaMerge/Index/IndexInfo.h create mode 100644 dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp create mode 100644 dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.h create mode 100644 dbms/src/Storages/DeltaMerge/LocalIndexerScheduler_fwd.h create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_vector_index.cpp create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index_utils.h create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_local_indexer_scheduler.cpp create mode 100644 dbms/src/Storages/System/StorageSystemDTLocalIndexes.cpp create mode 100644 dbms/src/Storages/System/StorageSystemDTLocalIndexes.h diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index c89497b8173..6cc25ab694f 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -54,6 +54,7 @@ M(DT_SnapshotOfReadRaw) \ M(DT_SnapshotOfSegmentSplit) \ M(DT_SnapshotOfSegmentMerge) \ + M(DT_SnapshotOfSegmentIngestIndex) \ M(DT_SnapshotOfSegmentIngest) \ M(DT_SnapshotOfDeltaMerge) \ M(DT_SnapshotOfDeltaCompact) \ diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1679e089d2a..e84c205f94c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -174,6 +175,7 @@ struct ContextShared PageStorageRunMode storage_run_mode = PageStorageRunMode::ONLY_V3; DM::GlobalPageIdAllocatorPtr global_page_id_allocator; DM::GlobalStoragePoolPtr global_storage_pool; + DM::LocalIndexerSchedulerPtr global_local_indexer_scheduler; /// The PS instance available on Write Node. UniversalPageStorageServicePtr ps_write; @@ -1763,6 +1765,27 @@ DM::GlobalPageIdAllocatorPtr Context::getGlobalPageIdAllocator() const return shared->global_page_id_allocator; } +bool Context::initializeGlobalLocalIndexerScheduler(size_t pool_size, size_t memory_limit) +{ + auto lock = getLock(); + if (!shared->global_local_indexer_scheduler) + { + shared->global_local_indexer_scheduler + = std::make_shared(DM::LocalIndexerScheduler::Options{ + .pool_size = pool_size, + .memory_limit = memory_limit, + .auto_start = true, + }); + } + return true; +} + +DM::LocalIndexerSchedulerPtr Context::getGlobalLocalIndexerScheduler() const +{ + auto lock = getLock(); + return shared->global_local_indexer_scheduler; +} + bool Context::initializeGlobalStoragePoolIfNeed(const PathPool & path_pool) { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 686da5d4696..61aef450855 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -26,6 +26,8 @@ #include #include #include +#include +#include #include #include @@ -457,6 +459,9 @@ class Context bool initializeGlobalPageIdAllocator(); DM::GlobalPageIdAllocatorPtr getGlobalPageIdAllocator() const; + bool initializeGlobalLocalIndexerScheduler(size_t pool_size, size_t memory_limit); + DM::LocalIndexerSchedulerPtr getGlobalLocalIndexerScheduler() const; + bool initializeGlobalStoragePoolIfNeed(const PathPool & path_pool); DM::GlobalStoragePoolPtr getGlobalStoragePool() const; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 67436141222..888ee0990e6 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -980,12 +980,13 @@ int Server::main(const std::vector & /*args*/) if (storage_config.format_version != 0) { - if (storage_config.s3_config.isS3Enabled() && storage_config.format_version != STORAGE_FORMAT_V100.identifier) + if (storage_config.s3_config.isS3Enabled() && storage_config.format_version != STORAGE_FORMAT_V100.identifier + && storage_config.format_version != STORAGE_FORMAT_V101.identifier) { - LOG_WARNING(log, "'storage.format_version' must be set to 100 when S3 is enabled!"); + LOG_WARNING(log, "'storage.format_version' must be set to 100 or 101 when S3 is enabled!"); throw Exception( ErrorCodes::INVALID_CONFIG_PARAMETER, - "'storage.format_version' must be set to 100 when S3 is enabled!"); + "'storage.format_version' must be set to 100 or 101 when S3 is enabled!"); } setStorageFormat(storage_config.format_version); LOG_INFO(log, "Using format_version={} (explicit storage format detected).", storage_config.format_version); @@ -996,8 +997,8 @@ int Server::main(const std::vector & /*args*/) { // If the user does not explicitly set format_version in the config file but // enables S3, then we set up a proper format version to support S3. - setStorageFormat(STORAGE_FORMAT_V100.identifier); - LOG_INFO(log, "Using format_version={} (infer by S3 is enabled).", STORAGE_FORMAT_V100.identifier); + setStorageFormat(STORAGE_FORMAT_V101.identifier); + LOG_INFO(log, "Using format_version={} (infer by S3 is enabled).", STORAGE_FORMAT_V101.identifier); } else { @@ -1301,6 +1302,24 @@ int Server::main(const std::vector & /*args*/) settings.max_memory_usage_for_all_queries.getActualBytes(server_info.memory_info.capacity), settings.bytes_that_rss_larger_than_limit); + if (global_context->getSharedContextDisagg()->isDisaggregatedComputeMode()) + { + // No need to have local index scheduler. + } + else if (global_context->getSharedContextDisagg()->isDisaggregatedStorageMode()) + { + global_context->initializeGlobalLocalIndexerScheduler( + server_info.cpu_info.logical_cores * 8 / 10, + server_info.memory_info.capacity * 6 / 10); + } + else + { + // There could be compute tasks, reserve more memory for computes. + global_context->initializeGlobalLocalIndexerScheduler( + server_info.cpu_info.logical_cores * 4 / 10, + server_info.memory_info.capacity * 4 / 10); + } + /// PageStorage run mode has been determined above global_context->initializeGlobalPageIdAllocator(); if (!global_context->getSharedContextDisagg()->isDisaggregatedComputeMode()) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 24cfa05d0a7..98b5ef8cee1 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -59,6 +60,7 @@ #include #include + namespace ProfileEvents { extern const Event DMWriteBlock; @@ -212,6 +214,7 @@ DeltaMergeStore::DeltaMergeStore( const ColumnDefine & handle, bool is_common_handle_, size_t rowkey_column_size_, + IndexInfosPtr local_index_infos_, const Settings & settings_, ThreadPool * thread_pool) : global_context(db_context.getGlobalContext()) @@ -228,6 +231,7 @@ DeltaMergeStore::DeltaMergeStore( , background_pool(db_context.getBackgroundPool()) , blockable_background_pool(db_context.getBlockableBackgroundPool()) , next_gc_check_key(is_common_handle ? RowKeyValue::COMMON_HANDLE_MIN_KEY : RowKeyValue::INT_HANDLE_MIN_KEY) + , local_index_infos(std::move(local_index_infos_)) , log(Logger::get(fmt::format("keyspace={} table_id={}", keyspace_id_, physical_table_id_))) { replica_exist.store(has_replica); @@ -276,9 +280,10 @@ DeltaMergeStore::DeltaMergeStore( { auto task = [this, dm_context, segment_id] { auto segment = Segment::restoreSegment(log, *dm_context, segment_id); - std::lock_guard lock(read_write_mutex); - segments.emplace(segment->getRowKeyRange().getEnd(), segment); - id_to_segment.emplace(segment_id, segment); + { + std::unique_lock lock(read_write_mutex); + addSegment(lock, segment); + } }; wait_group->schedule(task); } @@ -291,9 +296,10 @@ DeltaMergeStore::DeltaMergeStore( while (segment_id != 0) { auto segment = Segment::restoreSegment(log, *dm_context, segment_id); - segments.emplace(segment->getRowKeyRange().getEnd(), segment); - id_to_segment.emplace(segment_id, segment); - + { + std::unique_lock lock(read_write_mutex); + addSegment(lock, segment); + } segment_id = segment->nextSegmentId(); } } @@ -310,6 +316,78 @@ DeltaMergeStore::DeltaMergeStore( LOG_INFO(log, "Restore DeltaMerge Store end, ps_run_mode={}", magic_enum::enum_name(page_storage_run_mode)); } +DeltaMergeStorePtr DeltaMergeStore::create( + Context & db_context, + bool data_path_contains_database_name, + const String & db_name_, + const String & table_name_, + KeyspaceID keyspace_id_, + TableID physical_table_id_, + bool has_replica, + const ColumnDefines & columns, + const ColumnDefine & handle, + bool is_common_handle_, + size_t rowkey_column_size_, + IndexInfosPtr local_index_infos_, + const Settings & settings_, + ThreadPool * thread_pool) +{ + auto * store = new DeltaMergeStore( + db_context, + data_path_contains_database_name, + db_name_, + table_name_, + keyspace_id_, + physical_table_id_, + has_replica, + columns, + handle, + is_common_handle_, + rowkey_column_size_, + local_index_infos_, + settings_, + thread_pool); + std::shared_ptr store_shared_ptr(store); + store_shared_ptr->checkAllSegmentsLocalIndex(); + return store_shared_ptr; +} + +std::unique_ptr DeltaMergeStore::createUnique( + Context & db_context, + bool data_path_contains_database_name, + const String & db_name_, + const String & table_name_, + KeyspaceID keyspace_id_, + TableID physical_table_id_, + bool has_replica, + const ColumnDefines & columns, + const ColumnDefine & handle, + bool is_common_handle_, + size_t rowkey_column_size_, + IndexInfosPtr local_index_infos_, + const Settings & settings_, + ThreadPool * thread_pool) +{ + auto * store = new DeltaMergeStore( + db_context, + data_path_contains_database_name, + db_name_, + table_name_, + keyspace_id_, + physical_table_id_, + has_replica, + columns, + handle, + is_common_handle_, + rowkey_column_size_, + local_index_infos_, + settings_, + thread_pool); + std::unique_ptr store_unique_ptr(store); + store_unique_ptr->checkAllSegmentsLocalIndex(); + return store_unique_ptr; +} + DeltaMergeStore::~DeltaMergeStore() { LOG_INFO(log, "Release DeltaMerge Store start"); @@ -381,16 +459,11 @@ void DeltaMergeStore::dropAllSegments(bool keep_first_segment) // The order to drop the meta and data of this segment doesn't matter, // Because there is no segment pointing to this segment, // so it won't be restored again even the drop process was interrupted by restart - segments.erase(segment_to_drop->getRowKeyRange().getEnd()); - id_to_segment.erase(segment_id_to_drop); + removeSegment(lock, segment_to_drop); if (previous_segment) { assert(new_previous_segment); - assert(previous_segment->segmentId() == new_previous_segment->segmentId()); - segments.erase(previous_segment->getRowKeyRange().getEnd()); - segments.emplace(new_previous_segment->getRowKeyRange().getEnd(), new_previous_segment); - id_to_segment.erase(previous_segment->segmentId()); - id_to_segment.emplace(new_previous_segment->segmentId(), new_previous_segment); + replaceSegment(lock, previous_segment, new_previous_segment); } auto drop_lock = segment_to_drop->mustGetUpdateLock(); segment_to_drop->abandon(*dm_context); @@ -433,6 +506,11 @@ void DeltaMergeStore::shutdown() return; LOG_TRACE(log, "Shutdown DeltaMerge start"); + + auto indexer_scheulder = global_context.getGlobalLocalIndexerScheduler(); + RUNTIME_CHECK(indexer_scheulder != nullptr); + indexer_scheulder->dropTasks(keyspace_id, physical_table_id); + // Must shutdown storage path pool to make sure the DMFile remove callbacks // won't remove dmfiles unexpectly. path_pool->shutdown(); @@ -1924,6 +2002,8 @@ void DeltaMergeStore::applySchemaChanges(TableInfo & table_info) original_table_columns.swap(new_original_table_columns); store_columns.swap(new_store_columns); + // TODO(local index): There could be some local indexes added/dropped after DDL + std::atomic_store(&original_table_header, std::make_shared(toEmptyBlock(original_table_columns))); } @@ -2170,8 +2250,7 @@ void DeltaMergeStore::createFirstSegment(DM::DMContext & dm_context) RowKeyRange::newAll(is_common_handle, rowkey_column_size), segment_id, 0); - segments.emplace(first_segment->getRowKeyRange().getEnd(), first_segment); - id_to_segment.emplace(segment_id, first_segment); + addSegment(lock, first_segment); } } // namespace DM diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index e5c67ddef39..49fd1e16cdf 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -173,7 +174,26 @@ struct StoreStats UInt64 background_tasks_length = 0; }; -class DeltaMergeStore : private boost::noncopyable +struct LocalIndexStats +{ + String column_name{}; + UInt64 column_id{}; + String index_kind{}; + + UInt64 rows_stable_indexed{}; // Total rows + UInt64 rows_stable_not_indexed{}; // Total rows + UInt64 rows_delta_indexed{}; // Total rows + UInt64 rows_delta_not_indexed{}; // Total rows +}; +using LocalIndexesStats = std::vector; + + +class DeltaMergeStore; +using DeltaMergeStorePtr = std::shared_ptr; + +class DeltaMergeStore + : private boost::noncopyable + , public std::enable_shared_from_this { public: friend class ::DB::DM::tests::DeltaMergeStoreTest; @@ -259,8 +279,11 @@ class DeltaMergeStore : private boost::noncopyable BackgroundTask nextTask(bool is_heavy, const LoggerPtr & log_); }; +private: + // Let the constructor be private, so that we can control the creation of DeltaMergeStore. + // Please use DeltaMergeStore::create to create a DeltaMergeStore DeltaMergeStore( - Context & db_context, // + Context & db_context, bool data_path_contains_database_name, const String & db_name, const String & table_name_, @@ -271,8 +294,43 @@ class DeltaMergeStore : private boost::noncopyable const ColumnDefine & handle, bool is_common_handle_, size_t rowkey_column_size_, + IndexInfosPtr local_index_infos_, const Settings & settings_ = EMPTY_SETTINGS, ThreadPool * thread_pool = nullptr); + +public: + static DeltaMergeStorePtr create( + Context & db_context, + bool data_path_contains_database_name, + const String & db_name, + const String & table_name_, + KeyspaceID keyspace_id_, + TableID physical_table_id_, + bool has_replica, + const ColumnDefines & columns, + const ColumnDefine & handle, + bool is_common_handle_, + size_t rowkey_column_size_, + IndexInfosPtr local_index_infos_, + const Settings & settings_ = EMPTY_SETTINGS, + ThreadPool * thread_pool = nullptr); + + static std::unique_ptr createUnique( + Context & db_context, + bool data_path_contains_database_name, + const String & db_name, + const String & table_name_, + KeyspaceID keyspace_id_, + TableID physical_table_id_, + bool has_replica, + const ColumnDefines & columns, + const ColumnDefine & handle, + bool is_common_handle_, + size_t rowkey_column_size_, + IndexInfosPtr local_index_infos_, + const Settings & settings_ = EMPTY_SETTINGS, + ThreadPool * thread_pool = nullptr); + ~DeltaMergeStore(); void setUpBackgroundTask(const DMContextPtr & dm_context); @@ -502,6 +560,7 @@ class DeltaMergeStore : private boost::noncopyable StoreStats getStoreStats(); SegmentsStats getSegmentsStats(); + LocalIndexesStats getLocalIndexStats(); bool isCommonHandle() const { return is_common_handle; } size_t getRowKeyColumnSize() const { return rowkey_column_size; } @@ -648,6 +707,12 @@ class DeltaMergeStore : private boost::noncopyable MergeDeltaReason reason, SegmentSnapshotPtr segment_snap = nullptr); + void segmentEnsureStableIndex( + DMContext & dm_context, + const IndexInfosPtr & index_info, + const DMFiles & dm_files, + const String & source_segment_info); + /** * Ingest a DMFile into the segment, optionally causing a new segment being created. * @@ -756,11 +821,65 @@ class DeltaMergeStore : private boost::noncopyable const SegmentPtr & segment, ThreadType thread_type, InputType input_type); + + /** + * Segment update meta with new DMFiles. A lock must be provided, so that it is + * possible to update the meta for multiple segments all at once. + */ + SegmentPtr segmentUpdateMeta( + std::unique_lock & read_write_lock, + DMContext & dm_context, + const SegmentPtr & segment, + const DMFiles & new_dm_files); + + /** + * Remove the segment from the store's memory structure. + * Not protected by lock, should accquire lock before calling this function. + */ + void removeSegment(std::unique_lock &, const SegmentPtr & segment); + /** + * Add the segment to the store's memory structure. + * Not protected by lock, should accquire lock before calling this function. + */ + void addSegment(std::unique_lock &, const SegmentPtr & segment); + /** + * Replace the old segment with the new segment in the store's memory structure. + * New segment should have the same segment id as the old segment. + * Not protected by lock, should accquire lock before calling this function. + */ + void replaceSegment( + std::unique_lock &, + const SegmentPtr & old_segment, + const SegmentPtr & new_segment); + + /** + * Check whether there are new local indexes should be built for all segments. + */ + void checkAllSegmentsLocalIndex(); + + /** + * Ensure the segment has stable index. + * If the segment has no stable index, it will be built in background. + * Note: This function can not be called in constructor, since shared_from_this() is not available. + * + * @returns true if index is missing and a build task is added in background. + */ + bool segmentEnsureStableIndexAsync(const SegmentPtr & segment); + #ifndef DBMS_PUBLIC_GTEST private: #else public: #endif + /** + * Wait until the segment has stable index. + * If the index is ready or no need to build, it will return immediately. + * Only used for testing. + * + * @returns false if index is still missing after wait timed out. + */ + bool segmentWaitStableIndexReady(const SegmentPtr & segment) const; + void dropAllSegments(bool keep_first_segment); String getLogTracingId(const DMContext & dm_ctx); // Returns segment that contains start_key and whether 'segments' is empty. @@ -814,13 +933,35 @@ class DeltaMergeStore : private boost::noncopyable RowKeyValue next_gc_check_key; + // Some indexes are built in TiFlash locally. For example, Vector Index. + // Compares to the lightweight RoughSet Indexes, these indexes require lot + // of resources to build, so they will be built in separated background pool. + IndexInfosPtr local_index_infos; + + struct DMFileIDToSegmentIDs + { + public: + using Key = PageIdU64; // dmfile_id + using Value = std::unordered_set; // segment_ids + + void remove(SegmentPtr segment); + + void add(SegmentPtr segment); + + const Value & get(PageIdU64 dmfile_id) const; + + private: + std::unordered_map u_map; + }; + // dmfile_id -> segment_ids + // This map is not protected by lock, should be accessed under read_write_mutex. + DMFileIDToSegmentIDs dmfile_id_to_segment_ids; + // Synchronize between write threads and read threads. mutable std::shared_mutex read_write_mutex; LoggerPtr log; }; // namespace DM -using DeltaMergeStorePtr = std::shared_ptr; - } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp index 7995a3280ed..52e11e4c968 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_InternalSegment.cpp @@ -14,11 +14,18 @@ #include #include +#include #include +#include +#include #include #include #include +#include +#include +#include + namespace CurrentMetrics { @@ -31,12 +38,82 @@ extern const Metric DT_SnapshotOfSegmentSplit; extern const Metric DT_SnapshotOfSegmentMerge; extern const Metric DT_SnapshotOfDeltaMerge; extern const Metric DT_SnapshotOfSegmentIngest; +extern const Metric DT_SnapshotOfSegmentIngestIndex; } // namespace CurrentMetrics namespace DB { namespace DM { + +void DeltaMergeStore::DMFileIDToSegmentIDs::remove(SegmentPtr segment) +{ + RUNTIME_CHECK(segment != nullptr); + for (const auto & dmfile : segment->getStable()->getDMFiles()) + { + if (auto it = u_map.find(dmfile->fileId()); it != u_map.end()) + { + it->second.erase(segment->segmentId()); + } + } +} + +void DeltaMergeStore::DMFileIDToSegmentIDs::add(SegmentPtr segment) +{ + RUNTIME_CHECK(segment != nullptr); + for (const auto & dmfile : segment->getStable()->getDMFiles()) + { + u_map[dmfile->fileId()].insert(segment->segmentId()); + } +} + +const DeltaMergeStore::DMFileIDToSegmentIDs::Value & DeltaMergeStore::DMFileIDToSegmentIDs::get( + PageIdU64 dmfile_id) const +{ + static const Value empty; + if (auto it = u_map.find(dmfile_id); it != u_map.end()) + { + return it->second; + } + return empty; +} + +void DeltaMergeStore::removeSegment(std::unique_lock &, const SegmentPtr & segment) +{ + segments.erase(segment->getRowKeyRange().getEnd()); + id_to_segment.erase(segment->segmentId()); + dmfile_id_to_segment_ids.remove(segment); +} + +void DeltaMergeStore::addSegment(std::unique_lock &, const SegmentPtr & segment) +{ + RUNTIME_CHECK_MSG( + !segments.contains(segment->getRowKeyRange().getEnd()), + "Trying to add segment {} but there is a segment with the same key exists. Old segment must be removed " + "before adding new.", + segment->simpleInfo()); + segments[segment->getRowKeyRange().getEnd()] = segment; + id_to_segment[segment->segmentId()] = segment; + dmfile_id_to_segment_ids.add(segment); +} + +void DeltaMergeStore::replaceSegment( + std::unique_lock &, + const SegmentPtr & old_segment, + const SegmentPtr & new_segment) +{ + RUNTIME_CHECK( + old_segment->segmentId() == new_segment->segmentId(), + old_segment->segmentId(), + new_segment->segmentId()); + segments.erase(old_segment->getRowKeyRange().getEnd()); + dmfile_id_to_segment_ids.remove(old_segment); + + segments[new_segment->getRowKeyRange().getEnd()] = new_segment; + id_to_segment[new_segment->segmentId()] = new_segment; + dmfile_id_to_segment_ids.add(new_segment); +} + SegmentPair DeltaMergeStore::segmentSplit( DMContext & dm_context, const SegmentPtr & segment, @@ -179,14 +256,10 @@ SegmentPair DeltaMergeStore::segmentSplit( wbs.writeMeta(); segment->abandon(dm_context); - segments.erase(range.getEnd()); - id_to_segment.erase(segment->segmentId()); - - segments[new_left->getRowKeyRange().getEnd()] = new_left; - segments[new_right->getRowKeyRange().getEnd()] = new_right; - id_to_segment.emplace(new_left->segmentId(), new_left); - id_to_segment.emplace(new_right->segmentId(), new_right); + removeSegment(lock, segment); + addSegment(lock, new_left); + addSegment(lock, new_right); if constexpr (DM_RUN_CHECK) { @@ -224,6 +297,16 @@ SegmentPair DeltaMergeStore::segmentSplit( if constexpr (DM_RUN_CHECK) check(dm_context.db_context); + // For logical split, no new DMFile is created, new_left and new_right share the same DMFile with the old segment. + // Even if the index build process of the old segment is not finished, after it is finished, + // it will also trigger the new_left and new_right to bump the meta version. + // So there is no need to check the local index update for logical split. + if (!split_info.is_logical) + { + segmentEnsureStableIndexAsync(new_left); + segmentEnsureStableIndexAsync(new_right); + } + return {new_left, new_right}; } @@ -337,12 +420,10 @@ SegmentPtr DeltaMergeStore::segmentMerge( for (const auto & seg : ordered_segments) { seg->abandon(dm_context); - segments.erase(seg->getRowKeyRange().getEnd()); - id_to_segment.erase(seg->segmentId()); + removeSegment(lock, seg); } - segments.emplace(merged->getRowKeyRange().getEnd(), merged); - id_to_segment.emplace(merged->segmentId(), merged); + addSegment(lock, merged); if constexpr (DM_RUN_CHECK) merged->check(dm_context, "After segment merge"); @@ -364,9 +445,301 @@ SegmentPtr DeltaMergeStore::segmentMerge( if constexpr (DM_RUN_CHECK) check(dm_context.db_context); + segmentEnsureStableIndexAsync(merged); return merged; } +void DeltaMergeStore::checkAllSegmentsLocalIndex() +{ + if (!local_index_infos || local_index_infos->empty()) + { + return; + } + + LOG_INFO(log, "CheckAllSegmentsLocalIndex - Begin"); + + size_t segments_updated_meta = 0; + + auto dm_context = newDMContext(global_context, global_context.getSettingsRef(), "checkAllSegmentsLocalIndex"); + + // 1. Make all segments referencing latest meta version. + { + Stopwatch watch; + std::unique_lock lock(read_write_mutex); + + std::map latest_dmf_by_id; + for (const auto & [end, segment] : segments) + { + UNUSED(end); + for (const auto & dm_file : segment->getStable()->getDMFiles()) + { + auto & latest_dmf = latest_dmf_by_id[dm_file->fileId()]; + if (!latest_dmf || dm_file->metaVersion() > latest_dmf->metaVersion()) + // Note: pageId could be different. It is fine. + latest_dmf = dm_file; + } + } + for (const auto & [end, segment] : segments) + { + UNUSED(end); + for (const auto & dm_file : segment->getStable()->getDMFiles()) + { + auto & latest_dmf = latest_dmf_by_id.at(dm_file->fileId()); + if (dm_file->metaVersion() < latest_dmf->metaVersion()) + { + // Note: pageId could be different. It is fine, replaceStableMetaVersion will fix it. + auto update_result = segmentUpdateMeta(lock, *dm_context, segment, {latest_dmf}); + RUNTIME_CHECK(update_result != nullptr, segment->simpleInfo()); + ++segments_updated_meta; + } + } + } + LOG_INFO( + log, + "CheckAllSegmentsLocalIndex - Finish, updated_meta={}, elapsed={}", + segments_updated_meta, + watch.elapsedSeconds()); + } + + size_t segments_missing_indexes = 0; + + // 2. Trigger ensureStableIndex for all segments. + // There could be new segments between 1 and 2, which is fine. New segments + // will invoke ensureStableIndex at creation time. + { + // There must be a lock, because segments[] may be mutated. + // And one lock for all is fine, because segmentEnsureStableIndexAsync is non-blocking, it + // simply put tasks in the background. + std::shared_lock lock(read_write_mutex); + for (const auto & [end, segment] : segments) + { + UNUSED(end); + if (segmentEnsureStableIndexAsync(segment)) + ++segments_missing_indexes; + } + } + + LOG_INFO( + log, + "CheckAllSegmentsLocalIndex - Finish, segments_[updated_meta/missing_index]={}/{}", + segments_updated_meta, + segments_missing_indexes); +} + +bool DeltaMergeStore::segmentEnsureStableIndexAsync(const SegmentPtr & segment) +{ + RUNTIME_CHECK(segment != nullptr); + + // TODO(local index): There could be some indexes are built while some indexes is not yet built after DDL + if (!local_index_infos || local_index_infos->empty()) + return false; + + // No lock is needed, stable meta is immutable. + auto dm_files = segment->getStable()->getDMFiles(); + auto build_info = DMFileIndexWriter::getLocalIndexBuildInfo(local_index_infos, dm_files); + if (!build_info.indexes_to_build || build_info.indexes_to_build->empty()) + return false; + + auto store_weak_ptr = weak_from_this(); + auto tracing_id = fmt::format("segmentEnsureStableIndex<{}>", log->identifier()); + auto workload = [store_weak_ptr, build_info, dm_files, segment, tracing_id]() -> void { + auto store = store_weak_ptr.lock(); + if (store == nullptr) // Store is destroyed before the task is executed. + return; + auto dm_context = store->newDMContext( // + store->global_context, + store->global_context.getSettingsRef(), + tracing_id); + const auto source_segment_info = segment->simpleInfo(); + store->segmentEnsureStableIndex(*dm_context, build_info.indexes_to_build, dm_files, source_segment_info); + }; + + auto indexer_scheduler = global_context.getGlobalLocalIndexerScheduler(); + RUNTIME_CHECK(indexer_scheduler != nullptr); + indexer_scheduler->pushTask(LocalIndexerScheduler::Task{ + .keyspace_id = keyspace_id, + .table_id = physical_table_id, + .dmfile_ids = build_info.file_ids, + .request_memory = build_info.estimated_memory_bytes, + .workload = workload, + }); + return true; +} + +bool DeltaMergeStore::segmentWaitStableIndexReady(const SegmentPtr & segment) const +{ + RUNTIME_CHECK(segment != nullptr); + + // TODO(local index): There could be some indexes are built while some indexes is not yet built after DDL + if (!local_index_infos || local_index_infos->empty()) + return true; + + // No lock is needed, stable meta is immutable. + auto segment_id = segment->segmentId(); + auto dm_files = segment->getStable()->getDMFiles(); + auto build_info = DMFileIndexWriter::getLocalIndexBuildInfo(local_index_infos, dm_files); + if (!build_info.indexes_to_build || build_info.indexes_to_build->empty()) + return true; + + static constexpr size_t MAX_CHECK_TIME_SECONDS = 60; // 60s + Stopwatch watch; + while (watch.elapsedSeconds() < MAX_CHECK_TIME_SECONDS) + { + DMFilePtr dmfile; + { + std::shared_lock lock(read_write_mutex); + auto seg = id_to_segment.at(segment_id); + assert(!seg->getStable()->getDMFiles().empty()); + dmfile = seg->getStable()->getDMFiles()[0]; + } + if (!dmfile) + return false; // DMFile is not exist, return false + bool all_indexes_built = true; + for (const auto & index : *build_info.indexes_to_build) + { + auto col_id = index.column_id; + // The dmfile may be built before col_id is added. Skip build indexes for it + if (!dmfile->isColumnExist(col_id)) + continue; + + all_indexes_built = all_indexes_built && dmfile->getColumnStat(col_id).index_bytes > 0; + } + if (all_indexes_built) + return true; + std::this_thread::sleep_for(std::chrono::milliseconds(100)); // 0.1s + } + + return false; +} + +SegmentPtr DeltaMergeStore::segmentUpdateMeta( + std::unique_lock & read_write_lock, + DMContext & dm_context, + const SegmentPtr & segment, + const DMFiles & new_dm_files) +{ + if (!isSegmentValid(read_write_lock, segment)) + { + LOG_WARNING(log, "SegmentUpdateMeta - Give up because segment not valid, segment={}", segment->simpleInfo()); + return {}; + } + + auto lock = segment->mustGetUpdateLock(); + auto new_segment = segment->replaceStableMetaVersion(lock, dm_context, new_dm_files); + if (new_segment == nullptr) + { + LOG_WARNING( + log, + "SegmentUpdateMeta - Failed due to replace stableMeta failed, segment={}", + segment->simpleInfo()); + return {}; + } + + replaceSegment(read_write_lock, segment, new_segment); + + // Must not abandon old segment, because they share the same delta. + // segment->abandon(dm_context); + + if constexpr (DM_RUN_CHECK) + { + new_segment->check(dm_context, "After SegmentUpdateMeta"); + } + + LOG_INFO( + log, + "SegmentUpdateMeta - Finish, old_segment={} new_segment={}", + segment->simpleInfo(), + new_segment->simpleInfo()); + return new_segment; +} + +void DeltaMergeStore::segmentEnsureStableIndex( + DMContext & dm_context, + const IndexInfosPtr & index_info, + const DMFiles & dm_files, + const String & source_segment_info) +{ + // 1. Acquire a snapshot for PageStorage, and keep the snapshot until index is built. + // This helps keep DMFile valid during the index build process. + // We don't acquire a snapshot from the source_segment, because the source_segment + // may be abandoned at this moment. + // + // Note that we cannot simply skip the index building when seg is not valid any more, + // because segL and segR is still referencing them, consider this case: + // 1. seg=PhysicalSplit + // 2. Add CreateStableIndex(seg) to ThreadPool + // 3. segL, segR=LogicalSplit(seg) + // 4. CreateStableIndex(seg) + + auto storage_snapshot = std::make_shared( + *dm_context.storage_pool, + dm_context.getReadLimiter(), + dm_context.tracing_id, + /*snapshot_read*/ true); + + RUNTIME_CHECK(dm_files.size() == 1); // size > 1 is currently not supported. + const auto & dm_file = dm_files[0]; + + // 2. Check whether the DMFile has been referenced by any valid segment. + { + std::shared_lock lock(read_write_mutex); + auto segment_ids = dmfile_id_to_segment_ids.get(dm_file->fileId()); + if (segment_ids.empty()) + { + LOG_DEBUG( + log, + "EnsureStableIndex - Give up because no segment to update, source_segment={}", + source_segment_info); + return; + } + } + + LOG_INFO( + log, + "EnsureStableIndex - Begin building index, dm_files={} source_segment={}", + DMFile::info(dm_files), + source_segment_info); + + // 2. Build the index. + DMFileIndexWriter iw(DMFileIndexWriter::Options{ + .path_pool = path_pool, + .file_provider = dm_context.db_context.getFileProvider(), + .write_limiter = dm_context.getWriteLimiter(), + .disagg_ctx = dm_context.db_context.getSharedContextDisagg(), + .index_infos = index_info, + .dm_files = dm_files, + .db_context = dm_context.db_context, + .is_common_handle = dm_context.is_common_handle, + .rowkey_column_size = dm_context.rowkey_column_size, + }); + auto new_dmfiles = iw.build(); + RUNTIME_CHECK(!new_dmfiles.empty()); + + LOG_INFO( + log, + "EnsureStableIndex - Finish building index, dm_files={} source_segment={}", + DMFile::info(dm_files), + source_segment_info); + + // 3. Update the meta version of the segments to the latest one. + // To avoid logical split between step 2 and 3, get lastest segments to update again. + // If TiFlash crashes during updating the meta version, some segments' meta are updated and some are not. + // So after TiFlash restarts, we will update meta versions to latest versions again. + { + // We must acquire a single lock when updating multiple segments. + // Otherwise we may miss new segments. + std::unique_lock lock(read_write_mutex); + auto segment_ids = dmfile_id_to_segment_ids.get(dm_file->fileId()); + for (const auto & seg_id : segment_ids) + { + auto segment = id_to_segment[seg_id]; + auto new_segment = segmentUpdateMeta(lock, dm_context, segment, new_dmfiles); + // Expect update meta always success, because the segment must be valid and bump meta should succeed. + RUNTIME_CHECK_MSG(new_segment != nullptr, "Update meta failed for segment {}", segment->simpleInfo()); + } + } +} + SegmentPtr DeltaMergeStore::segmentMergeDelta( DMContext & dm_context, const SegmentPtr & segment, @@ -473,9 +846,9 @@ SegmentPtr DeltaMergeStore::segmentMergeDelta( SegmentPtr new_segment; { - std::unique_lock read_write_lock(read_write_mutex); + std::unique_lock lock(read_write_mutex); - if (!isSegmentValid(read_write_lock, segment)) + if (!isSegmentValid(lock, segment)) { LOG_DEBUG( log, @@ -491,14 +864,7 @@ SegmentPtr DeltaMergeStore::segmentMergeDelta( wbs.writeMeta(); - - // The instance of PKRange::End is closely linked to instance of PKRange. So we cannot reuse it. - // Replace must be done by erase + insert. - segments.erase(segment->getRowKeyRange().getEnd()); - id_to_segment.erase(segment->segmentId()); - - segments[new_segment->getRowKeyRange().getEnd()] = new_segment; - id_to_segment[new_segment->segmentId()] = new_segment; + replaceSegment(lock, segment, new_segment); segment->abandon(dm_context); @@ -522,6 +888,7 @@ SegmentPtr DeltaMergeStore::segmentMergeDelta( if constexpr (DM_RUN_CHECK) check(dm_context.db_context); + segmentEnsureStableIndexAsync(new_segment); return new_segment; } @@ -603,8 +970,7 @@ SegmentPtr DeltaMergeStore::segmentIngestData( RUNTIME_CHECK(segment->segmentId() == new_segment->segmentId(), segment->info(), new_segment->info()); segment->abandon(dm_context); - segments[segment->getRowKeyRange().getEnd()] = new_segment; - id_to_segment[segment->segmentId()] = new_segment; + replaceSegment(lock, segment, new_segment); LOG_INFO( log, @@ -632,6 +998,7 @@ SegmentPtr DeltaMergeStore::segmentIngestData( if constexpr (DM_RUN_CHECK) check(dm_context.db_context); + segmentEnsureStableIndexAsync(new_segment); return new_segment; } @@ -681,8 +1048,7 @@ SegmentPtr DeltaMergeStore::segmentDangerouslyReplaceDataFromCheckpoint( wbs.writeMeta(); segment->abandon(dm_context); - segments[segment->getRowKeyRange().getEnd()] = new_segment; - id_to_segment[segment->segmentId()] = new_segment; + replaceSegment(lock, segment, new_segment); LOG_INFO(log, "ReplaceData - Finish, old_segment={} new_segment={}", segment->info(), new_segment->info()); } @@ -692,6 +1058,7 @@ SegmentPtr DeltaMergeStore::segmentDangerouslyReplaceDataFromCheckpoint( if constexpr (DM_RUN_CHECK) check(dm_context.db_context); + segmentEnsureStableIndexAsync(new_segment); return new_segment; } diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp index 411d09a9f22..12e1ab15d62 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore_Statistics.cpp @@ -192,6 +192,60 @@ SegmentsStats DeltaMergeStore::getSegmentsStats() return stats; } +LocalIndexesStats DeltaMergeStore::getLocalIndexStats() +{ + std::shared_lock lock(read_write_mutex); + + if (!local_index_infos || local_index_infos->empty()) + return {}; + + LocalIndexesStats stats; + for (const auto & index_info : *local_index_infos) + { + LocalIndexStats index_stats; + index_stats.column_id = index_info.column_id; + index_stats.column_name = index_info.column_name; + index_stats.index_kind = "HNSW"; // TODO: Support more. + + for (const auto & [handle, segment] : segments) + { + UNUSED(handle); + + // Currently Delta is always not indexed. + index_stats.rows_delta_not_indexed + += segment->getDelta()->getRows(); // TODO: More precisely count column bytes instead. + + const auto & stable = segment->getStable(); + bool is_stable_indexed = true; + for (const auto & dmfile : stable->getDMFiles()) + { + if (!dmfile->isColumnExist(index_info.column_id)) + continue; // Regard as indexed, because column does not need any index + + auto column_stat = dmfile->getColumnStat(index_info.column_id); + + if (column_stat.index_bytes == 0 && column_stat.data_bytes > 0) + { + is_stable_indexed = false; + break; + } + } + + if (is_stable_indexed) + { + index_stats.rows_stable_indexed += stable->getRows(); + } + else + { + index_stats.rows_stable_not_indexed += stable->getRows(); + } + } + + stats.emplace_back(index_stats); + } + + return stats; +} } // namespace DM } // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp index bf932ba3a5a..aa69f7d9570 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp @@ -56,6 +56,16 @@ String DMFile::ngcPath() const return getNGCPath(parentPath(), fileId(), getStatus()); } +String DMFile::info(const DMFiles & files) +{ + String s; + for (const auto & file : files) + s += fmt::format("dmf_{}(v={}),", file->fileId(), file->metaVersion()); + if (!s.empty()) + s.erase(s.length() - 1); + return s; +} + DMFilePtr DMFile::create( UInt64 file_id, const String & parent_path, @@ -100,7 +110,6 @@ DMFilePtr DMFile::create( // since the NGC file is a file under the folder. // FIXME : this should not use PageUtils. PageUtil::touchFile(new_dmfile->ngcPath()); - return new_dmfile; } @@ -154,6 +163,7 @@ DMFilePtr DMFile::restore( STORAGE_FORMAT_CURRENT.dm_file); dmfile->meta->read(file_provider, read_meta_mode); } + return dmfile; } @@ -448,7 +458,7 @@ std::vector DMFile::listFilesForUpload() const return fnames; } -void DMFile::switchToRemote(const S3::DMFileOID & oid) +void DMFile::switchToRemote(const S3::DMFileOID & oid) const { RUNTIME_CHECK(useMetaV2()); RUNTIME_CHECK(getStatus() == DMFileStatus::READABLE); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h index 044a28b3e2a..7c2f983c496 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h @@ -68,6 +68,8 @@ class DMFile : private boost::noncopyable const DMFileMeta::ReadMode & read_meta_mode, UInt32 meta_version); + static String info(const DMFiles & dm_files); + struct ListOptions { // Only return the DTFiles id list that can be GC @@ -128,7 +130,7 @@ class DMFile : private boost::noncopyable const std::unordered_set & getColumnIndices() const { return meta->column_indices; } // only used in gtest - void clearPackProperties() { meta->pack_properties.clear_property(); } + void clearPackProperties() const { meta->pack_properties.clear_property(); } const ColumnStat & getColumnStat(ColId col_id) const { @@ -170,10 +172,12 @@ class DMFile : private boost::noncopyable } bool useMetaV2() const { return meta->format_version == DMFileFormat::V3; } + std::vector listFilesForUpload() const; - void switchToRemote(const S3::DMFileOID & oid); + void switchToRemote(const S3::DMFileOID & oid) const; UInt32 metaVersion() const { return meta->metaVersion(); } + UInt32 bumpMetaVersion() const { return meta->bumpMetaVersion(); } #ifndef DBMS_PUBLIC_GTEST private: @@ -269,10 +273,10 @@ class DMFile : private boost::noncopyable return IDataType::getFileNameForStream(DB::toString(col_id), substream); } - void addPack(const DMFileMeta::PackStat & pack_stat) { meta->pack_stats.push_back(pack_stat); } + void addPack(const DMFileMeta::PackStat & pack_stat) const { meta->pack_stats.push_back(pack_stat); } DMFileStatus getStatus() const { return meta->status; } - void setStatus(DMFileStatus status_) { meta->status = status_; } + void setStatus(DMFileStatus status_) const { meta->status = status_; } void finalize(); @@ -286,7 +290,7 @@ class DMFile : private boost::noncopyable friend class DMFileV3IncrementWriter; friend class DMFileWriter; - friend class DMFileWriterRemote; + friend class DMFileIndexWriter; friend class DMFileReader; friend class DMFilePackFilter; friend class DMFileBlockInputStreamBuilder; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp new file mode 100644 index 00000000000..6d957a01f79 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp @@ -0,0 +1,229 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::DM +{ + +DMFileIndexWriter::LocalIndexBuildInfo DMFileIndexWriter::getLocalIndexBuildInfo( + const IndexInfosPtr & index_infos, + const DMFiles & dm_files) +{ + assert(index_infos != nullptr); + static constexpr double VECTOR_INDEX_SIZE_FACTOR = 1.2; + + LocalIndexBuildInfo build; + build.indexes_to_build = std::make_shared(); + build.file_ids.reserve(dm_files.size()); + for (const auto & dmfile : dm_files) + { + bool any_new_index_build = false; + for (const auto & index : *index_infos) + { + auto col_id = index.column_id; + // The dmfile may be built before col_id is added. Skip build indexes for it + if (!dmfile->isColumnExist(col_id)) + continue; + + if (dmfile->getColumnStat(col_id).index_bytes > 0) + continue; + + any_new_index_build = true; + + auto col_stat = dmfile->getColumnStat(col_id); + build.indexes_to_build->emplace_back(index); + build.estimated_memory_bytes += col_stat.serialized_bytes * VECTOR_INDEX_SIZE_FACTOR; + } + + if (any_new_index_build) + { + build.file_ids.emplace_back(dmfile->fileId()); + } + } + + build.file_ids.shrink_to_fit(); + return build; +} + +void DMFileIndexWriter::buildIndexForFile(const DMFilePtr & dm_file_mutable) const +{ + const auto column_defines = dm_file_mutable->getColumnDefines(); + const auto del_cd_iter = std::find_if(column_defines.cbegin(), column_defines.cend(), [](const ColumnDefine & cd) { + return cd.id == TAG_COLUMN_ID; + }); + RUNTIME_CHECK_MSG( + del_cd_iter != column_defines.cend(), + "Cannot find del_mark column, file={}", + dm_file_mutable->path()); + + // read_columns are: DEL_MARK, COL_A, COL_B, ... + // index_builders are: COL_A, COL_B, ... + + ColumnDefines read_columns{*del_cd_iter}; + read_columns.reserve(options.index_infos->size() + 1); + + std::vector index_builders; + index_builders.reserve(options.index_infos->size()); + + // The caller should avoid building index for the same column multiple times. + for (const auto & index_info : *options.index_infos) + { + const auto cd_iter = std::find_if(column_defines.cbegin(), column_defines.cend(), [&](const auto & cd) { + return cd.id == index_info.column_id; + }); + RUNTIME_CHECK_MSG( + cd_iter != column_defines.cend(), + "Cannot find column_id={} in file={}", + index_info.column_id, + dm_file_mutable->path()); + + // Index already built. We don't allow. The caller should filter away, + RUNTIME_CHECK(dm_file_mutable->getColumnStat(index_info.column_id).index_bytes == 0, index_info.column_id); + + read_columns.push_back(*cd_iter); + index_builders.push_back(VectorIndexBuilder::create(index_info.index_definition)); + } + + if (index_builders.empty()) + { + // No index to build. + return; + } + + DMFileV3IncrementWriter::Options iw_options{ + .dm_file = dm_file_mutable, + .file_provider = options.file_provider, + .write_limiter = options.write_limiter, + .path_pool = options.path_pool, + .disagg_ctx = options.disagg_ctx, + }; + auto iw = DMFileV3IncrementWriter::create(iw_options); + + // TODO: Maybe using DMFileReader directly is better because it doesn't need db_context. + DMFileBlockInputStreamBuilder read_stream_builder(options.db_context); + auto scan_context = std::make_shared(); + + // Note: We use range::newAll to build index for all data in dmfile, because the index is file-level. + auto read_stream = read_stream_builder.build( + dm_file_mutable, + read_columns, + {RowKeyRange::newAll(options.is_common_handle, options.rowkey_column_size)}, + scan_context); + + // Read all blocks and build index + while (true) + { + auto block = read_stream->read(); + if (!block) + break; + + RUNTIME_CHECK(block.columns() == read_columns.size()); + RUNTIME_CHECK(block.getByPosition(0).column_id == TAG_COLUMN_ID); + + auto del_mark_col = block.safeGetByPosition(0).column; + RUNTIME_CHECK(del_mark_col != nullptr); + const auto * del_mark = static_cast *>(del_mark_col.get()); + RUNTIME_CHECK(del_mark != nullptr); + + for (size_t col_idx = 0, col_idx_max = index_builders.size(); col_idx < col_idx_max; ++col_idx) + { + const auto & index_builder = index_builders[col_idx]; + const auto & col_with_type_and_name = block.safeGetByPosition(col_idx + 1); + RUNTIME_CHECK(col_with_type_and_name.column_id == read_columns[col_idx + 1].id); + const auto & col = col_with_type_and_name.column; + index_builder->addBlock(*col, del_mark); + } + } + + // Write down the index + for (size_t col_idx = 0, col_idx_max = index_builders.size(); col_idx < col_idx_max; col_idx++) + { + const auto & index_builder = index_builders[col_idx]; + const auto & cd = read_columns[col_idx + 1]; + + // Save index and update column stats + auto callback = [&](const IDataType::SubstreamPath & substream_path) -> void { + if (IDataType::isNullMap(substream_path) || IDataType::isArraySizes(substream_path)) + return; + + const auto stream_name = DMFile::getFileNameBase(cd.id, substream_path); + const auto index_file_name = colIndexFileName(stream_name); + const auto index_path = iw->localPath() + "/" + index_file_name; + index_builder->save(index_path); + + auto & col_stat = dm_file_mutable->meta->getColumnStats().at(cd.id); + col_stat.index_bytes = Poco::File(index_path).getSize(); + // Memorize what kind of vector index it is, so that we can correctly restore it when reading. + col_stat.vector_index.emplace(); + col_stat.vector_index->set_index_kind(tipb::VectorIndexKind_Name(index_builder->definition->kind)); + col_stat.vector_index->set_distance_metric( + tipb::VectorDistanceMetric_Name(index_builder->definition->distance_metric)); + col_stat.vector_index->set_dimensions(index_builder->definition->dimension); + + iw->include(index_file_name); + }; + cd.type->enumerateStreams(callback); + } + + dm_file_mutable->meta->bumpMetaVersion(); + iw->finalize(); // Note: There may be S3 uploads here. +} + +DMFiles DMFileIndexWriter::build() const +{ + // Create a clone of existing DMFile instances by using DMFile::restore, + // because later we will mutate some fields and persist these mutations. + DMFiles cloned_dm_files{}; + + auto delegate = options.path_pool->getStableDiskDelegator(); + for (const auto & dm_file : options.dm_files) + { + if (!options.disagg_ctx || !options.disagg_ctx->remote_data_store) + { + RUNTIME_CHECK(dm_file->parentPath() == delegate.getDTFilePath(dm_file->fileId())); + } + + auto new_dmfile = DMFile::restore( + options.file_provider, + dm_file->fileId(), + dm_file->pageId(), + dm_file->parentPath(), + DMFileMeta::ReadMode::all(), + dm_file->metaVersion()); + + cloned_dm_files.push_back(new_dmfile); + } + + for (const auto & cloned_dmfile : cloned_dm_files) + { + buildIndexForFile(cloned_dmfile); + // TODO: including the new index bytes in the file size. + // auto res = dm_context.path_pool->getStableDiskDelegator().updateDTFileSize( + // new_dmfile->fileId(), + // new_dmfile->getBytesOnDisk()); + // RUNTIME_CHECK_MSG(res, "update dt file size failed, path={}", new_dmfile->path()); + } + + return cloned_dm_files; +} + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.h b/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.h new file mode 100644 index 00000000000..1eedff4507b --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.h @@ -0,0 +1,84 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ +class StoragePathPool; +using StoragePathPoolPtr = std::shared_ptr; +class WriteLimiter; +using WriteLimiterPtr = std::shared_ptr; +} // namespace DB + +namespace DB::DM +{ +class DMFile; +using DMFilePtr = std::shared_ptr; +using DMFiles = std::vector; +} // namespace DB::DM + +namespace DB::DM +{ + +class DMFileIndexWriter +{ +public: + struct LocalIndexBuildInfo + { + std::vector file_ids; + size_t estimated_memory_bytes = 0; + IndexInfosPtr indexes_to_build; + }; + + static LocalIndexBuildInfo getLocalIndexBuildInfo(const IndexInfosPtr & index_infos, const DMFiles & dm_files); + + struct Options + { + const StoragePathPoolPtr path_pool; + const FileProviderPtr file_provider; + const WriteLimiterPtr write_limiter; + const SharedContextDisaggPtr disagg_ctx; + const IndexInfosPtr index_infos; + const DMFiles dm_files; + const Context & db_context; + const bool is_common_handle; + const size_t rowkey_column_size; + }; + + explicit DMFileIndexWriter(const Options & options) + : logger(Logger::get()) + , options(options) + {} + + // Note: You cannot call build() multiple times, as duplicate meta version will result in exceptions. + // TODO: Add a better guard. + DMFiles build() const; + +private: + void buildIndexForFile(const DMFilePtr & dm_file_mutable) const; + +private: + const LoggerPtr logger; + const Options options; +}; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.cpp index d2beb957d89..4e4f380515a 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.cpp @@ -15,14 +15,12 @@ #include #include #include -#include #include #include #include -#include -#include #include + namespace DB::DM { @@ -148,15 +146,6 @@ void DMFileV3IncrementWriter::writeAndIncludeMetaFile() // to ensure file's integrity. auto meta_file_path_for_write = meta_file_path + ".tmp"; - // Just a protection. We don't allow overwriting meta file. - { - auto existing_file = Poco::File(meta_file_path); - RUNTIME_CHECK_MSG( // - !existing_file.exists(), - "Meta file already exists, file={}", - meta_file_path); - } - auto meta_file = std::make_unique( options.file_provider, meta_file_path_for_write, // Must not use meta->metaPath(), because DMFile may be a S3 DMFile diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.h b/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.h index 0886d04c44f..1f5262ef70a 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileV3IncrementWriter.h @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include @@ -35,10 +35,8 @@ using StoragePathPoolPtr = std::shared_ptr; namespace DB::DM { - class DMFile; using DMFilePtr = std::shared_ptr; - } // namespace DB::DM namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp index fbac9278471..9510a3aa81e 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp @@ -61,14 +61,12 @@ DMFileWriter::DMFileWriter( for (auto & cd : write_columns) { - if (cd.vector_index) - RUNTIME_CHECK(VectorIndexBuilder::isSupportedType(*cd.type)); - // TODO: currently we only generate index for Integers, Date, DateTime types, and this should be configurable by user. /// for handle column always generate index auto type = removeNullable(cd.type); bool do_index = cd.id == EXTRA_HANDLE_COLUMN_ID || type->isInteger() || type->isDateOrDateTime(); - addStreams(cd.id, cd.type, do_index, cd.vector_index); + + addStreams(cd.id, cd.type, do_index); dmfile->meta->getColumnStats().emplace(cd.id, ColumnStat{cd.id, cd.type, /*avg_size=*/0}); } } @@ -117,11 +115,7 @@ DMFileWriter::WriteBufferFromFileBasePtr DMFileWriter::createPackStatsFile() options.max_compress_block_size); } -void DMFileWriter::addStreams( - ColId col_id, - DataTypePtr type, - bool do_index, - TiDB::VectorIndexDefinitionPtr do_vector_index) +void DMFileWriter::addStreams(ColId col_id, DataTypePtr type, bool do_index) { auto callback = [&](const IDataType::SubstreamPath & substream_path) { const auto stream_name = DMFile::getFileNameBase(col_id, substream_path); @@ -134,8 +128,7 @@ void DMFileWriter::addStreams( options.max_compress_block_size, file_provider, write_limiter, - do_index && substream_can_index, - (do_vector_index && substream_can_index) ? do_vector_index : nullptr); + do_index && substream_can_index); column_streams.emplace(stream_name, std::move(stream)); }; @@ -225,9 +218,6 @@ void DMFileWriter::writeColumn( (col_id == EXTRA_HANDLE_COLUMN_ID || col_id == TAG_COLUMN_ID) ? nullptr : del_mark); } - if (stream->vector_index) - stream->vector_index->addBlock(column, del_mark); - /// There could already be enough data to compress into the new block. if (stream->compressed_buf->offset() >= options.min_compress_block_size) stream->compressed_buf->next(); @@ -349,24 +339,6 @@ void DMFileWriter::finalizeColumn(ColId col_id, DataTypePtr type) buffer->next(); } - if (stream->vector_index && !is_empty_file) - { - // Vector index files are always not written into the merged file - // because we want to allow to be mmaped by the usearch. - - const auto index_name = dmfile->colIndexPath(stream_name); - stream->vector_index->save(index_name); - col_stat.index_bytes = Poco::File(index_name).getSize(); - - // Memorize what kind of vector index it is, so that we can correctly restore it when reading. - col_stat.vector_index.emplace(); - col_stat.vector_index->set_index_kind( - tipb::VectorIndexKind_Name(stream->vector_index->definition->kind)); - col_stat.vector_index->set_distance_metric( - tipb::VectorDistanceMetric_Name(stream->vector_index->definition->distance_metric)); - col_stat.vector_index->set_dimensions(stream->vector_index->definition->dimension); - } - // write mark into merged_file_writer if (!is_empty_file) { @@ -474,11 +446,6 @@ void DMFileWriter::finalizeColumn(ColId col_id, DataTypePtr type) #endif } } - - if (stream->vector_index) - { - RUNTIME_CHECK_MSG(false, "Vector index is not compatible with V1 and V2 format"); - } } }; type->enumerateStreams(callback, {}); diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h index ce1ea398352..e84023b36d9 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.h @@ -54,8 +54,7 @@ class DMFileWriter size_t max_compress_block_size, FileProviderPtr & file_provider, const WriteLimiterPtr & write_limiter_, - bool do_index, - TiDB::VectorIndexDefinitionPtr do_vector_index) + bool do_index) : plain_file(WriteBufferByFileProviderBuilder( dmfile->getConfiguration().has_value(), file_provider, @@ -73,7 +72,6 @@ class DMFileWriter : std::unique_ptr( new CompressedWriteBuffer(*plain_file, compression_settings))) , minmaxes(do_index ? std::make_shared(*type) : nullptr) - , vector_index(do_vector_index ? VectorIndexBuilder::create(do_vector_index) : nullptr) { if (!dmfile->useMetaV2()) { @@ -100,7 +98,6 @@ class DMFileWriter WriteBufferPtr compressed_buf; MinMaxIndexPtr minmaxes; - VectorIndexBuilderPtr vector_index; MarksInCompressedFilePtr marks; @@ -162,7 +159,7 @@ class DMFileWriter /// Add streams with specified column id. Since a single column may have more than one Stream, /// for example Nullable column has a NullMap column, we would track them with a mapping /// FileNameBase -> Stream. - void addStreams(ColId col_id, DataTypePtr type, bool do_index, TiDB::VectorIndexDefinitionPtr do_vector_index); + void addStreams(ColId col_id, DataTypePtr type, bool do_index); WriteBufferFromFileBasePtr createMetaFile(); WriteBufferFromFileBasePtr createMetaV2File(); diff --git a/dbms/src/Storages/DeltaMerge/Index/IndexInfo.h b/dbms/src/Storages/DeltaMerge/Index/IndexInfo.h new file mode 100644 index 00000000000..5837c432251 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/Index/IndexInfo.h @@ -0,0 +1,41 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +namespace DB::DM +{ + +enum class IndexType +{ + Vector = 1, +}; + +struct IndexInfo +{ + IndexType type; + ColumnID column_id; + String column_name; + // Now we only support vector index. + // In the future, we may support more types of indexes, using std::variant. + TiDB::VectorIndexDefinitionPtr index_definition; +}; + +using IndexInfos = std::vector; +using IndexInfosPtr = std::shared_ptr; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp b/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp new file mode 100644 index 00000000000..beb0a7b396a --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.cpp @@ -0,0 +1,422 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +namespace DB::DM +{ + +LocalIndexerScheduler::LocalIndexerScheduler(const Options & options) + : logger(Logger::get()) + , pool(std::make_unique(options.pool_size, options.pool_size, options.pool_size + 1)) + , pool_max_memory_limit(options.memory_limit) + , pool_current_memory(0) +{ + // QueueSize = PoolSize+1, because our scheduler will try to schedule next task + // right before the current task is finished. + + LOG_INFO( + logger, + "Initialized LocalIndexerScheduler, pool_size={}, memory_limit_mb={:.1f}", + options.pool_size, + static_cast(options.memory_limit) / 1024 / 1024); + + if (options.auto_start) + start(); +} + +LocalIndexerScheduler::~LocalIndexerScheduler() +{ + LOG_INFO(logger, "LocalIndexerScheduler is destroying. Waiting scheduler and tasks to finish..."); + + // First quit the scheduler. Don't schedule more tasks. + is_shutting_down = true; + { + std::unique_lock lock(mutex); + scheduler_need_wakeup = true; + scheduler_notifier.notify_all(); + } + + if (is_started) + scheduler_thread.join(); + + // Then wait all running tasks to finish. + pool.reset(); + + LOG_INFO(logger, "LocalIndexerScheduler is destroyed"); +} + +void LocalIndexerScheduler::start() +{ + if (is_started) + return; + + scheduler_thread = std::thread([this]() { schedulerLoop(); }); + is_started = true; +} + +void LocalIndexerScheduler::waitForFinish() +{ + while (true) + { + std::unique_lock lock(mutex); + if (all_tasks_count == 0 && running_tasks_count == 0) + return; + on_finish_notifier.wait(lock); + } +} + +void LocalIndexerScheduler::pushTask(const Task & task) +{ + if (pool_max_memory_limit > 0 && task.request_memory > pool_max_memory_limit) + throw Exception(fmt::format( + "Requests memory exceeds limit (request={} limit={})", + task.request_memory, + pool_max_memory_limit)); + + std::unique_lock lock(mutex); + + const auto internal_task = std::make_shared(InternalTask{ + .user_task = task, + .created_at = Stopwatch(), + .scheduled_at = Stopwatch{}, // Not scheduled + }); + + // Whether task is ready is undertermined. It can be changed any time + // according to current running tasks. + // The scheduler will find a better place for this task when meeting it. + ready_tasks[task.keyspace_id][task.table_id].emplace_back(internal_task); + all_tasks_count++; + + scheduler_need_wakeup = true; + scheduler_notifier.notify_all(); +} + +size_t LocalIndexerScheduler::dropTasks(KeyspaceID keyspace_id, TableID table_id) +{ + size_t dropped_tasks = 0; + + std::unique_lock lock(mutex); + auto it = ready_tasks.find(keyspace_id); + if (it != ready_tasks.end()) + { + auto & tasks_by_table = it->second; + auto table_it = tasks_by_table.find(table_id); + if (table_it != tasks_by_table.end()) + { + dropped_tasks += table_it->second.size(); + tasks_by_table.erase(table_it); + } + if (tasks_by_table.empty()) + ready_tasks.erase(it); + } + for (auto it = unready_tasks.begin(); it != unready_tasks.end();) + { + if ((*it)->user_task.keyspace_id == keyspace_id && (*it)->user_task.table_id == table_id) + { + it = unready_tasks.erase(it); + dropped_tasks++; + } + else + { + it++; + } + } + + LOG_INFO(logger, "Removed {} tasks, keyspace_id={} table_id={}", dropped_tasks, keyspace_id, table_id); + + return dropped_tasks; +} + +bool LocalIndexerScheduler::isTaskReady(std::unique_lock & lock, const InternalTaskPtr & task) +{ + UNUSED(lock); + + for (const auto & dmfile_id : task->user_task.dmfile_ids) + { + auto unique_dmfile_id = UniqueDMFileID{ + .keyspace_id = task->user_task.keyspace_id, + .table_id = task->user_task.table_id, + .dmfile_id = dmfile_id, + }; + if (adding_index_dmfile_id_set.find(unique_dmfile_id) != adding_index_dmfile_id_set.end()) + return false; + } + return true; +} + +void LocalIndexerScheduler::taskOnSchedule(std::unique_lock & lock, const InternalTaskPtr & task) +{ + UNUSED(lock); + + for (const auto & dmfile_id : task->user_task.dmfile_ids) + { + auto unique_dmfile_id = UniqueDMFileID{ + .keyspace_id = task->user_task.keyspace_id, + .table_id = task->user_task.table_id, + .dmfile_id = dmfile_id, + }; + auto [it, inserted] = adding_index_dmfile_id_set.insert(unique_dmfile_id); + RUNTIME_CHECK(inserted); + UNUSED(it); + } + + LOG_DEBUG( // + logger, + "Start LocalIndex task, keyspace_id={} table_id={} dmfile_ids={} " + "memory_[this/total/limit]_mb={:.1f}/{:.1f}/{:.1f} all_tasks={}", + task->user_task.keyspace_id, + task->user_task.table_id, + task->user_task.dmfile_ids, + static_cast(task->user_task.request_memory) / 1024 / 1024, + static_cast(pool_current_memory) / 1024 / 1024, + static_cast(pool_max_memory_limit) / 1024 / 1024, + all_tasks_count); + + // No need to update unready_tasks here, because we will update unready_tasks + // when iterating the full list. +} + +void LocalIndexerScheduler::taskOnFinish(std::unique_lock & lock, const InternalTaskPtr & task) +{ + UNUSED(lock); + + for (const auto & dmfile_id : task->user_task.dmfile_ids) + { + auto unique_dmfile_id = UniqueDMFileID{ + .keyspace_id = task->user_task.keyspace_id, + .table_id = task->user_task.table_id, + .dmfile_id = dmfile_id, + }; + auto erased = adding_index_dmfile_id_set.erase(unique_dmfile_id); + RUNTIME_CHECK(erased == 1, erased); + } + + moveBackReadyTasks(lock); + + auto elapsed_since_create = task->created_at.elapsedSeconds(); + auto elapsed_since_schedule = task->scheduled_at.elapsedSeconds(); + + LOG_DEBUG( // + logger, + "Finish LocalIndex task, keyspace_id={} table_id={} dmfile_ids={} " + "memory_[this/total/limit]_mb={:.1f}/{:.1f}/{:.1f} " + "[schedule/task]_cost_sec={:.1f}/{:.1f}", + task->user_task.keyspace_id, + task->user_task.table_id, + task->user_task.dmfile_ids, + static_cast(task->user_task.request_memory) / 1024 / 1024, + static_cast(pool_current_memory) / 1024 / 1024, + static_cast(pool_max_memory_limit) / 1024 / 1024, + elapsed_since_create - elapsed_since_schedule, + elapsed_since_schedule); +} + +void LocalIndexerScheduler::moveBackReadyTasks(std::unique_lock & lock) +{ + UNUSED(lock); + + for (auto it = unready_tasks.begin(); it != unready_tasks.end();) + { + auto & task = *it; + if (isTaskReady(lock, task)) + { + ready_tasks[task->user_task.keyspace_id][task->user_task.table_id].emplace_back(task); + it = unready_tasks.erase(it); + } + else + { + it++; + } + } +} + +bool LocalIndexerScheduler::tryAddTaskToPool(std::unique_lock & lock, const InternalTaskPtr & task) +{ + UNUSED(lock); + + // Memory limit reached + if (pool_max_memory_limit > 0 && pool_current_memory + task->user_task.request_memory > pool_max_memory_limit) + { + return false; + } + + auto real_job = [task, this]() { + SCOPE_EXIT({ + std::unique_lock lock(mutex); + pool_current_memory -= task->user_task.request_memory; + running_tasks_count--; + taskOnFinish(lock, task); + on_finish_notifier.notify_all(); + + scheduler_need_wakeup = true; + scheduler_notifier.notify_all(); + }); + + task->scheduled_at.start(); + + try + { + task->user_task.workload(); + } + catch (...) + { + // TODO: We should ensure ADD INDEX task is retried somewhere else? + tryLogCurrentException( + logger, + fmt::format( + "LocalIndexScheduler meet exception when running task: keyspace_id={} table_id={}", + task->user_task.keyspace_id, + task->user_task.table_id)); + } + }; + + RUNTIME_CHECK(pool); + bool ok = pool->trySchedule(real_job); + if (!ok) + // Concurrent task limit reached + return false; + + running_tasks_count++; + pool_current_memory += task->user_task.request_memory; + taskOnSchedule(lock, task); + + return true; +} + +LocalIndexerScheduler::ScheduleResult LocalIndexerScheduler::scheduleNextTask(std::unique_lock & lock) +{ + if (ready_tasks.empty()) + return ScheduleResult::FAIL_NO_TASK; + + // Find the keyspace ID which is just > last_schedule_keyspace_id. + auto keyspace_it = ready_tasks.upper_bound(last_schedule_keyspace_id); + if (keyspace_it == ready_tasks.end()) + keyspace_it = ready_tasks.begin(); + const KeyspaceID keyspace_id = keyspace_it->first; + + auto & tasks_by_table = keyspace_it->second; + RUNTIME_CHECK(!tasks_by_table.empty()); + + TableID last_schedule_table_id = -1; + if (last_schedule_table_id_by_ks.find(keyspace_id) != last_schedule_table_id_by_ks.end()) + last_schedule_table_id = last_schedule_table_id_by_ks[keyspace_id]; + + auto table_it = tasks_by_table.upper_bound(last_schedule_table_id); + if (table_it == tasks_by_table.end()) + table_it = tasks_by_table.begin(); + const TableID table_id = table_it->first; + + auto & tasks = table_it->second; + RUNTIME_CHECK(!tasks.empty()); + auto task_it = tasks.begin(); + auto task = *task_it; + + auto remove_current_task = [&]() { + tasks.erase(task_it); + if (tasks.empty()) + { + tasks_by_table.erase(table_it); + if (tasks_by_table.empty()) + { + ready_tasks.erase(keyspace_id); + last_schedule_table_id_by_ks.erase(keyspace_id); + } + } + }; + + if (!isTaskReady(lock, task)) + { + // The task is not ready. Move it to unready_tasks. + unready_tasks.emplace_back(task); + remove_current_task(); + + LOG_DEBUG( + logger, + "LocalIndex task is not ready, will try again later when it is ready. " + "keyspace_id={} table_id={} dmfile_ids={}", + task->user_task.keyspace_id, + task->user_task.table_id, + task->user_task.dmfile_ids); + + // Let the caller retry. At next retry, we will continue using this + // Keyspace+Table and try next task. + return ScheduleResult::RETRY; + } + + auto ok = tryAddTaskToPool(lock, task); + if (!ok) + // The pool is full. May be memory limit reached or concurrent task limit reached. + // We will not try any more tasks. + // At next retry, we will continue using this Keyspace+Table and try next task. + return ScheduleResult::FAIL_FULL; + + last_schedule_table_id_by_ks[keyspace_id] = table_id; + last_schedule_keyspace_id = keyspace_id; + remove_current_task(); + all_tasks_count--; + + return ScheduleResult::OK; +} + +void LocalIndexerScheduler::schedulerLoop() +{ + while (true) + { + if (is_shutting_down) + return; + + std::unique_lock lock(mutex); + scheduler_notifier.wait(lock, [&] { return scheduler_need_wakeup || is_shutting_down; }); + scheduler_need_wakeup = false; + + try + { + while (true) + { + if (is_shutting_down) + return; + + auto result = scheduleNextTask(lock); + if (result == ScheduleResult::FAIL_FULL) + { + // Cannot schedule task any more, start to wait + break; + } + else if (result == ScheduleResult::FAIL_NO_TASK) + { + // No task to schedule, start to wait + break; + } + else if (result == ScheduleResult::RETRY) + { + // Retry schedule again + } + else if (result == ScheduleResult::OK) + { + // Task is scheduled, continue to schedule next task + } + } + } + catch (...) + { + // Catch all exceptions to avoid the scheduler thread to be terminated. + // We should log the exception here. + tryLogCurrentException(logger, __PRETTY_FUNCTION__); + } + } +} +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.h b/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.h new file mode 100644 index 00000000000..6ae490d1c1b --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler.h @@ -0,0 +1,208 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + +namespace DB::DM +{ + +// Note: this scheduler is global in the TiFlash instance. +class LocalIndexerScheduler +{ +public: + struct Task + { + // Note: The scheduler will try to schedule farely according to keyspace_id and table_id. + KeyspaceID keyspace_id; + TableID table_id; + + // Used for the scheduler to avoid concurrently adding index for the same DMFile. + std::vector dmfile_ids; + + // Used for the scheduler to control the maximum requested memory usage. + size_t request_memory; + + // The actual index setup workload. + // The scheduler does not care about the workload. + ThreadPool::Job workload; + }; + + struct Options + { + size_t pool_size = 1; + size_t memory_limit = 0; // 0 = unlimited + bool auto_start = true; + }; + +private: + struct InternalTask + { + const Task user_task; + Stopwatch created_at{}; + Stopwatch scheduled_at{}; + }; + + using InternalTaskPtr = std::shared_ptr; + +public: + static LocalIndexerSchedulerPtr create(const Options & options) + { + return std::make_shared(options); + } + + explicit LocalIndexerScheduler(const Options & options); + + ~LocalIndexerScheduler(); + + /** + * @brief Start the scheduler. In some tests we need to start scheduler + * after some tasks are pushed. + */ + void start(); + + /** + * @brief Blocks until there is no tasks remaining in the queue and there is no running tasks. + * Should be only used in tests. + */ + void waitForFinish(); + + /** + * @brief Push a task to the pool. The task may not be scheduled immediately. + */ + void pushTask(const Task & task); + + /** + * @brief Drop all tasks matching specified keyspace id and table id. + */ + size_t dropTasks(KeyspaceID keyspace_id, TableID table_id); + +private: + struct UniqueDMFileID + { + // We could use DMFile path as well, but this should be faster. + + KeyspaceID keyspace_id; + TableID table_id; + PageIdU64 dmfile_id; + + bool operator==(const UniqueDMFileID & other) const + { + return keyspace_id == other.keyspace_id // + && table_id == other.table_id // + && dmfile_id == other.dmfile_id; + } + }; + + struct UniqueDMFileIDHasher + { + std::size_t operator()(const UniqueDMFileID & id) const + { + using boost::hash_combine; + using boost::hash_value; + + std::size_t seed = 0; + hash_combine(seed, hash_value(id.keyspace_id)); + hash_combine(seed, hash_value(id.table_id)); + hash_combine(seed, hash_value(id.dmfile_id)); + return seed; + } + }; + + // The set of DMFiles that are currently adding index. + // There maybe multiple threads trying to add index for the same DMFile. For example, + // after logical split two segments share the same DMFile, so that adding index for the two segments + // could result in adding the same index for the same DMFile. It's just a waste of resource. + std::unordered_set adding_index_dmfile_id_set; + + bool isTaskReady(std::unique_lock & lock, const InternalTaskPtr & task); + + void taskOnSchedule(std::unique_lock & lock, const InternalTaskPtr & task); + + void taskOnFinish(std::unique_lock & lock, const InternalTaskPtr & task); + + void moveBackReadyTasks(std::unique_lock & lock); + +private: + bool is_started = false; + std::thread scheduler_thread; + + /// Try to add a task to the pool. Returns false if the pool is full + /// (for example, reaches concurrent task limit or memory limit). + /// When pool is full, we will not try to schedule any more tasks at this moment. + /// + /// Actually there could be possibly small tasks to schedule when + /// reaching memory limit, but this will cause the scheduler tend to + /// only schedule small tasks, keep large tasks starving under + /// heavy pressure. + bool tryAddTaskToPool(std::unique_lock & lock, const InternalTaskPtr & task); + + KeyspaceID last_schedule_keyspace_id = 0; + std::map last_schedule_table_id_by_ks; + + enum class ScheduleResult + { + RETRY, + FAIL_FULL, + FAIL_NO_TASK, + OK, + }; + + ScheduleResult scheduleNextTask(std::unique_lock & lock); + + void schedulerLoop(); + +private: + std::mutex mutex; + + const LoggerPtr logger; + + /// The thread pool for creating indices in the background. + std::unique_ptr pool; + /// The current memory usage of the pool. It is not accurate and the memory + /// is determined when task is adding to the pool. + const size_t pool_max_memory_limit; + size_t pool_current_memory = 0; + + size_t all_tasks_count = 0; // ready_tasks + unready_tasks + /// Schedule farely according to keyspace_id, and then according to table_id. + std::map>> ready_tasks{}; + /// When the scheduler will stop waiting and try to schedule again? + /// 1. When a new task is added (and pool is not full) + /// 2. When a pool task is finished + std::condition_variable scheduler_notifier; + bool scheduler_need_wakeup = false; // Avoid false wake-ups. + + /// Notified when one task is finished. + std::condition_variable on_finish_notifier; + size_t running_tasks_count = 0; + + /// Some tasks cannot be scheduled at this moment. For example, its DMFile + /// is used in another index building task. These tasks are extracted + /// from ready_tasks and put into unready_tasks. + std::list unready_tasks{}; + + std::atomic is_shutting_down = false; +}; + + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler_fwd.h b/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler_fwd.h new file mode 100644 index 00000000000..1f77cf2b321 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/LocalIndexerScheduler_fwd.h @@ -0,0 +1,26 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace DB::DM +{ + +class LocalIndexerScheduler; + +using LocalIndexerSchedulerPtr = std::shared_ptr; + +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/Segment.cpp b/dbms/src/Storages/DeltaMerge/Segment.cpp index 1181760d0c3..027b1e448af 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.cpp +++ b/dbms/src/Storages/DeltaMerge/Segment.cpp @@ -36,8 +36,9 @@ #include #include #include -// #include +#include #include +#include #include #include #include @@ -1300,50 +1301,16 @@ SegmentPtr Segment::replaceStableMetaVersion( DMContext & dm_context, const DMFiles & new_stable_files) { - auto current_stable_files_str = [&] { - FmtBuffer fmt_buf; - fmt_buf.append('['); - fmt_buf.joinStr( - stable->getDMFiles().begin(), - stable->getDMFiles().end(), - [](const DMFilePtr & file, FmtBuffer & fb) { - fb.fmtAppend("dmf_{}(v{})", file->fileId(), file->metaVersion()); - }, - ","); - fmt_buf.append(']'); - return fmt_buf.toString(); - }; - - auto new_stable_files_str = [&] { - FmtBuffer fmt_buf; - fmt_buf.append('['); - fmt_buf.joinStr( - new_stable_files.begin(), - new_stable_files.end(), - [](const DMFilePtr & file, FmtBuffer & fb) { - fb.fmtAppend("dmf_{}(v{})", file->fileId(), file->metaVersion()); - }, - ","); - fmt_buf.append(']'); - return fmt_buf.toString(); - }; - - LOG_DEBUG( - log, - "ReplaceStableMetaVersion - Begin, current_stable={} new_stable={}", - current_stable_files_str(), - new_stable_files_str()); - - // Ensure new stable files have the same DMFile ID as the old stable files. + // Ensure new stable files have the same DMFile ID and Page ID as the old stable files. // We only allow changing meta version when calling this function. if (new_stable_files.size() != stable->getDMFiles().size()) { LOG_WARNING( log, - "ReplaceStableMetaVersion - Fail, stable files count mismatch, current_stable={} new_stable={}", - current_stable_files_str(), - new_stable_files_str()); + "ReplaceStableMetaVersion - Failed due to stable mismatch, current_stable={} new_stable={}", + DMFile::info(stable->getDMFiles()), + DMFile::info(new_stable_files)); return {}; } for (size_t i = 0; i < new_stable_files.size(); i++) @@ -1352,17 +1319,45 @@ SegmentPtr Segment::replaceStableMetaVersion( { LOG_WARNING( log, - "ReplaceStableMetaVersion - Fail, stable files mismatch, current_stable={} new_stable={}", - current_stable_files_str(), - new_stable_files_str()); + "ReplaceStableMetaVersion - Failed due to stable mismatch, current_stable={} " + "new_stable={}", + DMFile::info(stable->getDMFiles()), + DMFile::info(new_stable_files)); return {}; } } WriteBatches wbs(*dm_context.storage_pool, dm_context.getWriteLimiter()); + DMFiles new_dm_files; + new_dm_files.reserve(new_stable_files.size()); + const auto & current_stable_files = stable->getDMFiles(); + for (size_t file_idx = 0; file_idx < new_stable_files.size(); ++file_idx) + { + const auto & new_file = new_stable_files[file_idx]; + const auto & current_file = current_stable_files[file_idx]; + RUNTIME_CHECK(new_file->fileId() == current_file->fileId()); + if (new_file->pageId() != current_file->pageId()) + { + // Allow pageId being different. We will restore using a correct pageId + // because this function is supposed to only update meta version. + auto new_dmfile = DMFile::restore( + dm_context.db_context.getFileProvider(), + new_file->fileId(), + current_file->pageId(), + new_file->parentPath(), + DMFileMeta::ReadMode::all(), + new_file->metaVersion()); + new_dm_files.push_back(new_dmfile); + } + else + { + new_dm_files.push_back(new_file); + } + } + auto new_stable = std::make_shared(stable->getId()); - new_stable->setFiles(new_stable_files, rowkey_range, &dm_context); + new_stable->setFiles(new_dm_files, rowkey_range, &dm_context); new_stable->saveMeta(wbs.meta); auto new_me = std::make_shared( // @@ -1377,8 +1372,11 @@ SegmentPtr Segment::replaceStableMetaVersion( wbs.writeAll(); - LOG_DEBUG(log, "ReplaceStableMetaVersion - Finish, new_stable_files={}", new_stable_files_str()); - + LOG_DEBUG( + log, + "ReplaceStableMetaVersion - Finish, new_stable={} old_stable={}", + DMFile::info(new_stable_files), + DMFile::info(stable->getDMFiles())); return new_me; } @@ -2362,9 +2360,10 @@ void Segment::placeDeltaIndex(DMContext & dm_context, const SegmentSnapshotPtr & String Segment::simpleInfo() const { return fmt::format( - "", + "", segment_id, epoch, + stable->getDMFilesString(), rowkey_range.toDebugString(), hasAbandoned() ? " abandoned=true" : ""); } diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index 8eb5502ef01..87cd0df84af 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -742,7 +742,6 @@ class Segment const ColumnDefines & read_columns, const StableValueSpacePtr & stable); - #ifndef DBMS_PUBLIC_GTEST private: #else diff --git a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp index 936b48a9a60..8f745758b53 100644 --- a/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/StableValueSpace.cpp @@ -335,12 +335,7 @@ size_t StableValueSpace::getDMFilesBytes() const String StableValueSpace::getDMFilesString() { - String s; - for (auto & file : files) - s += "dmf_" + DB::toString(file->fileId()) + ","; - if (!s.empty()) - s.erase(s.length() - 1); - return s; + return DMFile::info(files); } void StableValueSpace::enableDMFilesGC(DMContext & context) 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 74113b88e95..91608478974 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 @@ -261,7 +261,7 @@ try { new_cols = DMTestEnv::getDefaultColumns(); ColumnDefine handle_column_define = (*new_cols)[0]; - new_store = std::make_shared( + new_store = DeltaMergeStore::create( *db_context, false, "test", @@ -273,6 +273,7 @@ try handle_column_define, false, 1, + nullptr, DeltaMergeStore::Settings()); auto block = DMTestEnv::prepareSimpleWriteBlock(0, 100, false); new_store->write(*db_context, db_context->getSettingsRef(), block); @@ -3144,7 +3145,7 @@ class DeltaMergeStoreMergeDeltaBySegmentTest void setupDMStore() { auto cols = DMTestEnv::getDefaultColumns(pk_type); - store = std::make_shared( + store = DeltaMergeStore::create( *db_context, false, "test", @@ -3156,6 +3157,7 @@ class DeltaMergeStoreMergeDeltaBySegmentTest (*cols)[0], pk_type == DMTestEnv::PkType::CommonHandle, 1, + nullptr, DeltaMergeStore::Settings()); dm_context = store->newDMContext( *db_context, @@ -3169,7 +3171,7 @@ class DeltaMergeStoreMergeDeltaBySegmentTest DMContextPtr dm_context; UInt64 ps_ver{}; - DMTestEnv::PkType pk_type; + DMTestEnv::PkType pk_type{}; }; INSTANTIATE_TEST_CASE_P( diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp index 56fee2849ab..22a24e4649d 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_fast_add_peer.cpp @@ -155,7 +155,7 @@ class DeltaMergeStoreTestFastAddPeer ColumnDefine handle_column_define = (*cols)[0]; - DeltaMergeStorePtr s = std::make_shared( + DeltaMergeStorePtr s = DeltaMergeStore::create( *db_context, false, "test", @@ -167,6 +167,7 @@ class DeltaMergeStoreTestFastAddPeer handle_column_define, is_common_handle, rowkey_column_size, + nullptr, DeltaMergeStore::Settings()); return s; } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_test_basic.h b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_test_basic.h index 58a111334e9..3231b1f9a6a 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_test_basic.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_test_basic.h @@ -67,7 +67,7 @@ class DeltaMergeStoreTest : public DB::base::TiFlashStorageTestBasic ColumnDefine handle_column_define = (*cols)[0]; - DeltaMergeStorePtr s = std::make_shared( + DeltaMergeStorePtr s = DeltaMergeStore::create( *db_context, false, "test", @@ -79,6 +79,7 @@ class DeltaMergeStoreTest : public DB::base::TiFlashStorageTestBasic handle_column_define, is_common_handle, rowkey_column_size, + nullptr, DeltaMergeStore::Settings()); return s; } @@ -159,7 +160,7 @@ class DeltaMergeStoreRWTest ColumnDefine handle_column_define = (*cols)[0]; - DeltaMergeStorePtr s = std::make_shared( + DeltaMergeStorePtr s = DeltaMergeStore::create( *db_context, false, "test", @@ -171,6 +172,7 @@ class DeltaMergeStoreRWTest handle_column_define, is_common_handle, rowkey_column_size, + nullptr, DeltaMergeStore::Settings()); return s; } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_vector_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_vector_index.cpp new file mode 100644 index 00000000000..5d49358444f --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store_vector_index.cpp @@ -0,0 +1,569 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include + + +namespace DB::DM::tests +{ + +class DeltaMergeStoreVectorTest + : public DB::base::TiFlashStorageTestBasic + , public VectorIndexTestUtils +{ +public: + void SetUp() override + { + TiFlashStorageTestBasic::SetUp(); + store = reload(); + } + + DeltaMergeStorePtr reload() + { + TiFlashStorageTestBasic::reload(); + auto cols = DMTestEnv::getDefaultColumns(); + cols->push_back(cdVec()); + + ColumnDefine handle_column_define = (*cols)[0]; + + DeltaMergeStorePtr s = DeltaMergeStore::create( + *db_context, + false, + "test", + "t_100", + NullspaceID, + 100, + true, + *cols, + handle_column_define, + false, + 1, + indexInfo(), + DeltaMergeStore::Settings()); + return s; + } + + void write(size_t num_rows_write) + { + String sequence = fmt::format("[0, {})", num_rows_write); + Block block; + { + block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); + // Add a column of vector for test + block.insert(colVecFloat32(sequence, vec_column_name, vec_column_id)); + } + store->write(*db_context, db_context->getSettingsRef(), block); + } + + void read(const RowKeyRange & range, const PushDownFilterPtr & filter, const ColumnWithTypeAndName & out) + { + auto in = store->read( + *db_context, + db_context->getSettingsRef(), + {cdVec()}, + {range}, + /* num_streams= */ 1, + /* max_version= */ std::numeric_limits::max(), + filter, + std::vector{}, + 0, + TRACING_NAME, + /*keep_order=*/false)[0]; + ASSERT_INPUTSTREAM_COLS_UR( + in, + Strings({vec_column_name}), + createColumns({ + out, + })); + } + + void triggerMergeDelta() + { + std::vector all_segments; + { + std::shared_lock lock(store->read_write_mutex); + for (const auto & [_, segment] : store->id_to_segment) + all_segments.push_back(segment); + } + auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); + for (const auto & segment : all_segments) + ASSERT_TRUE( + store->segmentMergeDelta(*dm_context, segment, DeltaMergeStore::MergeDeltaReason::Manual) != nullptr); + } + + void waitStableIndexReady() + { + std::vector all_segments; + { + std::shared_lock lock(store->read_write_mutex); + for (const auto & [_, segment] : store->id_to_segment) + all_segments.push_back(segment); + } + for (const auto & segment : all_segments) + ASSERT_TRUE(store->segmentWaitStableIndexReady(segment)); + } + + void triggerMergeAllSegments() + { + auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); + std::vector segments_to_merge; + { + std::shared_lock lock(store->read_write_mutex); + for (const auto & [_, segment] : store->id_to_segment) + segments_to_merge.push_back(segment); + } + std::sort(segments_to_merge.begin(), segments_to_merge.end(), [](const auto & lhs, const auto & rhs) { + return lhs->getRowKeyRange().getEnd() < rhs->getRowKeyRange().getEnd(); + }); + auto new_segment = store->segmentMerge( + *dm_context, + segments_to_merge, + DeltaMergeStore::SegmentMergeReason::BackgroundGCThread); + ASSERT_TRUE(new_segment != nullptr); + } + +protected: + DeltaMergeStorePtr store; + + constexpr static const char * TRACING_NAME = "DeltaMergeStoreVectorTest"; +}; + +TEST_F(DeltaMergeStoreVectorTest, TestBasic) +try +{ + store = reload(); + + const size_t num_rows_write = 128; + + // write to store + write(num_rows_write); + + // trigger mergeDelta for all segments + triggerMergeDelta(); + + // check stable index has built for all segments + waitStableIndexReady(); + + const auto range = RowKeyRange::newAll(store->is_common_handle, store->rowkey_column_size); + + // read from store + { + read(range, EMPTY_FILTER, colVecFloat32("[0, 128)", vec_column_name, vec_column_id)); + } + + auto ann_query_info = std::make_shared(); + ann_query_info->set_column_id(vec_column_id); + ann_query_info->set_distance_metric(tipb::VectorDistanceMetric::L2); + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } +} +CATCH + +TEST_F(DeltaMergeStoreVectorTest, TestLogicalSplitAndMerge) +try +{ + store = reload(); + + const size_t num_rows_write = 128; + + // write to store + write(num_rows_write); + + // trigger mergeDelta for all segments + triggerMergeDelta(); + + // logical split + RowKeyRange left_segment_range; + { + SegmentPtr segment; + { + std::shared_lock lock(store->read_write_mutex); + segment = store->segments.begin()->second; + } + auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); + auto breakpoint = RowKeyValue::fromHandle(num_rows_write / 2); + const auto [left, right] = store->segmentSplit( + *dm_context, + segment, + DeltaMergeStore::SegmentSplitReason::ForIngest, + breakpoint, + DeltaMergeStore::SegmentSplitMode::Logical); + ASSERT_TRUE(left->rowkey_range.end == breakpoint); + ASSERT_TRUE(right->rowkey_range.start == breakpoint); + left_segment_range = RowKeyRange( + left->rowkey_range.start, + left->rowkey_range.end, + store->is_common_handle, + store->rowkey_column_size); + } + + // check stable index has built for all segments + waitStableIndexReady(); + + // read from store + { + read( + left_segment_range, + EMPTY_FILTER, + colVecFloat32(fmt::format("[0, {})", num_rows_write / 2), vec_column_name, vec_column_id)); + } + + auto ann_query_info = std::make_shared(); + ann_query_info->set_column_id(vec_column_id); + ann_query_info->set_distance_metric(tipb::VectorDistanceMetric::L2); + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(left_segment_range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({122.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(left_segment_range, filter, createVecFloat32Column({})); // FIXME: should be 63.0 + } + + // merge segment + triggerMergeAllSegments(); + + // check stable index has built for all segments + waitStableIndexReady(); + + auto range = RowKeyRange::newAll(store->is_common_handle, store->rowkey_column_size); + + // read from store + { + read(range, EMPTY_FILTER, colVecFloat32("[0, 128)", vec_column_name, vec_column_id)); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({122.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{122.0}})); + } +} +CATCH + +TEST_F(DeltaMergeStoreVectorTest, TestPhysicalSplitAndMerge) +try +{ + // Physical split is slow, so if we trigger mergeDelta and then physical split soon, + // the physical split is likely to fail since vector index building cause segment to be invalid. + + store = reload(); + + const size_t num_rows_write = 128; + + // write to store + write(num_rows_write); + + // trigger mergeDelta for all segments + triggerMergeDelta(); + + // physical split + auto physical_split = [&] { + SegmentPtr segment; + { + std::shared_lock lock(store->read_write_mutex); + segment = store->segments.begin()->second; + } + auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); + auto breakpoint = RowKeyValue::fromHandle(num_rows_write / 2); + return store->segmentSplit( + *dm_context, + segment, + DeltaMergeStore::SegmentSplitReason::ForIngest, + breakpoint, + DeltaMergeStore::SegmentSplitMode::Physical); + }; + + auto [left, right] = physical_split(); + if (left == nullptr && right == nullptr) + { + // check stable index has built for all segments first + waitStableIndexReady(); + // trigger physical split again + std::tie(left, right) = physical_split(); + } + + ASSERT_TRUE(left->rowkey_range.end == RowKeyValue::fromHandle(num_rows_write / 2)); + ASSERT_TRUE(right->rowkey_range.start == RowKeyValue::fromHandle(num_rows_write / 2)); + RowKeyRange left_segment_range = RowKeyRange( + left->rowkey_range.start, + left->rowkey_range.end, + store->is_common_handle, + store->rowkey_column_size); + + // check stable index has built for all segments + waitStableIndexReady(); + + // read from store + { + read( + left_segment_range, + EMPTY_FILTER, + colVecFloat32(fmt::format("[0, {})", num_rows_write / 2), vec_column_name, vec_column_id)); + } + + auto ann_query_info = std::make_shared(); + ann_query_info->set_column_id(vec_column_id); + ann_query_info->set_distance_metric(tipb::VectorDistanceMetric::L2); + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(left_segment_range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({122.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(left_segment_range, filter, createVecFloat32Column({{63.0}})); + } + + // merge segment + triggerMergeAllSegments(); + + // check stable index has built for all segments + waitStableIndexReady(); + + auto range = RowKeyRange::newAll(store->is_common_handle, store->rowkey_column_size); + + // read from store + { + read(range, EMPTY_FILTER, colVecFloat32("[0, 128)", vec_column_name, vec_column_id)); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({122.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{122.0}})); + } +} +CATCH + +TEST_F(DeltaMergeStoreVectorTest, TestIngestData) +try +{ + store = reload(); + + const size_t num_rows_write = 128; + + // write to store + write(num_rows_write); + + // Prepare DMFile + auto [dmfile_parent_path, file_id] = store->preAllocateIngestFile(); + ASSERT_FALSE(dmfile_parent_path.empty()); + DMFilePtr dmfile = DMFile::create( + file_id, + dmfile_parent_path, + std::make_optional(), + 128 * 1024, + 16 * 1024 * 1024, + DMFileFormat::V3); + { + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false); + // Add a column of vector for test + block.insert(colVecFloat32(fmt::format("[0, {})", num_rows_write), vec_column_name, vec_column_id)); + ColumnDefinesPtr cols = DMTestEnv::getDefaultColumns(); + cols->push_back(cdVec()); + auto stream = std::make_shared(*db_context, dmfile, *cols); + stream->writePrefix(); + stream->write(block, DMFileBlockOutputStream::BlockProperty{0, 0, 0, 0}); + stream->writeSuffix(); + } + auto page_id = dmfile->pageId(); + auto file_provider = db_context->getFileProvider(); + dmfile = DMFile::restore( + file_provider, + file_id, + page_id, + dmfile_parent_path, + DMFileMeta::ReadMode::all(), + /* meta_version= */ 0); + auto delegator = store->path_pool->getStableDiskDelegator(); + delegator.addDTFile(file_id, dmfile->getBytesOnDisk(), dmfile_parent_path); + + // Ingest data + { + // Ingest data into the first segment + auto segment = store->segments.begin()->second; + auto range = segment->getRowKeyRange(); + + auto dm_context = store->newDMContext(*db_context, db_context->getSettingsRef()); + auto new_segment = store->segmentIngestData(*dm_context, segment, dmfile, true); + ASSERT_TRUE(new_segment != nullptr); + } + + // check stable index has built for all segments + waitStableIndexReady(); + + auto range = RowKeyRange::newAll(store->is_common_handle, store->rowkey_column_size); + + // read from store + { + read(range, EMPTY_FILTER, colVecFloat32("[0, 128)", vec_column_name, vec_column_id)); + } + + auto ann_query_info = std::make_shared(); + ann_query_info->set_column_id(vec_column_id); + ann_query_info->set_distance_metric(tipb::VectorDistanceMetric::L2); + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } +} +CATCH + + +TEST_F(DeltaMergeStoreVectorTest, TestStoreRestore) +try +{ + store = reload(); + + const size_t num_rows_write = 128; + + // write to store + write(num_rows_write); + + // trigger mergeDelta for all segments + triggerMergeDelta(); + + // shutdown store + store->shutdown(); + + // restore store + store = reload(); + + // check stable index has built for all segments + waitStableIndexReady(); + + const auto range = RowKeyRange::newAll(store->is_common_handle, store->rowkey_column_size); + + // read from store + { + read(range, EMPTY_FILTER, colVecFloat32("[0, 128)", vec_column_name, vec_column_id)); + } + + auto ann_query_info = std::make_shared(); + ann_query_info->set_column_id(vec_column_id); + ann_query_info->set_distance_metric(tipb::VectorDistanceMetric::L2); + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.0})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } + + // read with ANN query + { + ann_query_info->set_top_k(1); + ann_query_info->set_ref_vec_f32(encodeVectorFloat32({2.1})); + + auto filter = std::make_shared(wrapWithANNQueryInfo(nullptr, ann_query_info)); + + read(range, filter, createVecFloat32Column({{2.0}})); + } +} +CATCH + +} // namespace DB::DM::tests 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 b20c1fd4ead..8ced751ffb5 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -114,7 +114,7 @@ bool checkMatch( Block block = genBlock(header, block_tuples); // max page id is only updated at restart, so we need recreate page v3 before recreate table - DeltaMergeStorePtr store = std::make_shared( + DeltaMergeStorePtr store = DeltaMergeStore::create( context, false, "test_database", @@ -125,7 +125,8 @@ bool checkMatch( table_columns, getExtraHandleColumnDefine(is_common_handle), is_common_handle, - 1); + 1, + nullptr); store->write(context, context.getSettingsRef(), block); store->flushCache(context, all_range); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_simple_pk_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_simple_pk_test_basic.cpp index 7245321d64a..9ae61382ed2 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_simple_pk_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_simple_pk_test_basic.cpp @@ -47,7 +47,7 @@ void SimplePKTestBasic::reload() auto cols = DMTestEnv::getDefaultColumns( is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID); - store = std::make_shared( + store = DeltaMergeStore::create( *db_context, false, "test", @@ -59,6 +59,7 @@ void SimplePKTestBasic::reload() (*cols)[0], is_common_handle, 1, + nullptr, DeltaMergeStore::Settings()); dm_context = store->newDMContext( *db_context, diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index.cpp index 72d792fbcea..dca7792b381 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index.cpp @@ -18,11 +18,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -35,12 +37,15 @@ #include #include +#include #include #include + namespace CurrentMetrics { extern const Metric DT_SnapshotOfRead; +extern const Metric DT_SnapshotOfSegmentIngestIndex; } // namespace CurrentMetrics namespace DB::FailPoints @@ -52,50 +57,6 @@ extern const char file_cache_fg_download_fail[]; namespace DB::DM::tests { -class VectorIndexTestUtils -{ -public: - const ColumnID vec_column_id = 100; - const String vec_column_name = "vec"; - - /// Create a column with values like [1], [2], [3], ... - /// Each value is a VectorFloat32 with exactly one dimension. - static ColumnWithTypeAndName colInt64(std::string_view sequence, const String & name = "", Int64 column_id = 0) - { - auto data = genSequence(sequence); - return createColumn(data, name, column_id); - } - - static ColumnWithTypeAndName colVecFloat32(std::string_view sequence, const String & name = "", Int64 column_id = 0) - { - auto data = genSequence(sequence); - std::vector data_in_array; - for (auto & v : data) - { - Array vec; - vec.push_back(static_cast(v)); - data_in_array.push_back(vec); - } - return createVecFloat32Column(data_in_array, name, column_id); - } - - static String encodeVectorFloat32(const std::vector & vec) - { - WriteBufferFromOwnString wb; - Array arr; - for (const auto & v : vec) - arr.push_back(static_cast(v)); - EncodeVectorFloat32(arr, wb); - return wb.str(); - } - - ColumnDefine cdVec() - { - // When used in read, no need to assign vector_index. - return ColumnDefine(vec_column_id, vec_column_name, tests::typeFromString("Array(Float32)")); - } -}; - class VectorIndexDMFileTest : public VectorIndexTestUtils , public DB::base::TiFlashStorageTestBasic @@ -107,12 +68,14 @@ class VectorIndexDMFileTest TiFlashStorageTestBasic::SetUp(); parent_path = TiFlashStorageTestBasic::getTemporaryPath(); - path_pool = std::make_shared( - db_context->getPathPool().withTable("test", "VectorIndexDMFileTest", false)); + path_pool = std::make_shared(db_context->getPathPool().withTable("test", "t1", false)); storage_pool = std::make_shared(*db_context, NullspaceID, /*ns_id*/ 100, *path_pool, "test.t1"); + auto delegator = path_pool->getStableDiskDelegator(); + auto paths = delegator.listPaths(); + RUNTIME_CHECK(paths.size() == 1); dm_file = DMFile::create( 1, - parent_path, + paths[0], std::make_optional(), 128 * 1024, 16 * 1024 * 1024, @@ -146,13 +109,36 @@ class VectorIndexDMFileTest auto file_id = dm_file->fileId(); auto page_id = dm_file->pageId(); auto file_provider = dbContext().getFileProvider(); - return DMFile::restore( + auto dmfile_parent_path = dm_file->parentPath(); + auto dmfile = DMFile::restore( file_provider, file_id, page_id, - parent_path, + dmfile_parent_path, DMFileMeta::ReadMode::all(), /* meta_version= */ 0); + auto delegator = path_pool->getStableDiskDelegator(); + delegator.addDTFile(1, dmfile->getBytesOnDisk(), dmfile_parent_path); + return dmfile; + } + + DMFilePtr buildIndex(TiDB::VectorIndexDefinition definition) + { + auto build_info = DMFileIndexWriter::getLocalIndexBuildInfo(indexInfo(definition), {dm_file}); + DMFileIndexWriter iw(DMFileIndexWriter::Options{ + .path_pool = path_pool, + .file_provider = dm_context->db_context.getFileProvider(), + .write_limiter = dm_context->getWriteLimiter(), + .disagg_ctx = dm_context->db_context.getSharedContextDisagg(), + .index_infos = build_info.indexes_to_build, + .dm_files = {dm_file}, + .db_context = dm_context->db_context, + .is_common_handle = dm_context->is_common_handle, + .rowkey_column_size = dm_context->rowkey_column_size, + }); + auto new_dmfiles = iw.build(); + assert(new_dmfiles.size() == 1); + return new_dmfiles[0]; } Context & dbContext() { return *db_context; } @@ -175,7 +161,7 @@ class VectorIndexDMFileTest // So we test it independently. bool test_only_vec_column = false; - ColumnsWithTypeAndName createColumnData(const ColumnsWithTypeAndName & columns) + ColumnsWithTypeAndName createColumnData(const ColumnsWithTypeAndName & columns) const { if (!test_only_vec_column) return columns; @@ -230,6 +216,7 @@ try } dm_file = restoreDMFile(); + dm_file = buildIndex(*vec_cd.vector_index); // Read with exact match { @@ -540,6 +527,7 @@ try } dm_file = restoreDMFile(); + dm_file = buildIndex(*vec_cd.vector_index); { auto ann_query_info = std::make_shared(); @@ -606,6 +594,7 @@ try } dm_file = restoreDMFile(); + dm_file = buildIndex(*vec_cd.vector_index); // Pack #0 is filtered out according to VecIndex { @@ -748,6 +737,7 @@ try } dm_file = restoreDMFile(); + dm_file = buildIndex(*vec_cd.vector_index); // Pack Filter using RowKeyRange { @@ -900,7 +890,7 @@ class VectorIndexSegmentTestBase bool test_only_vec_column = false; int pack_size = 10; - ColumnsWithTypeAndName createColumnData(const ColumnsWithTypeAndName & columns) + ColumnsWithTypeAndName createColumnData(const ColumnsWithTypeAndName & columns) const { if (!test_only_vec_column) return columns; @@ -1047,6 +1037,7 @@ try ingestDTFileIntoDelta(DELTA_MERGE_FIRST_SEGMENT_ID, 5, /* at */ 0, /* clear */ false); flushSegmentCache(DELTA_MERGE_FIRST_SEGMENT_ID); mergeSegmentDelta(DELTA_MERGE_FIRST_SEGMENT_ID); + ensureSegmentStableIndex(DELTA_MERGE_FIRST_SEGMENT_ID, indexInfo()); auto stream = annQuery(DELTA_MERGE_FIRST_SEGMENT_ID, createQueryColumns(), 1, {100.0}); assertStreamOut(stream, "[4, 5)"); @@ -1071,6 +1062,7 @@ try ingestDTFileIntoDelta(DELTA_MERGE_FIRST_SEGMENT_ID, 5, /* at */ 0, /* clear */ false); flushSegmentCache(DELTA_MERGE_FIRST_SEGMENT_ID); mergeSegmentDelta(DELTA_MERGE_FIRST_SEGMENT_ID); + ensureSegmentStableIndex(DELTA_MERGE_FIRST_SEGMENT_ID, indexInfo()); writeSegment(DELTA_MERGE_FIRST_SEGMENT_ID, 10, /* at */ 20); @@ -1101,6 +1093,7 @@ try ingestDTFileIntoDelta(DELTA_MERGE_FIRST_SEGMENT_ID, 10, /* at */ 20, /* clear */ false); flushSegmentCache(DELTA_MERGE_FIRST_SEGMENT_ID); mergeSegmentDelta(DELTA_MERGE_FIRST_SEGMENT_ID); + ensureSegmentStableIndex(DELTA_MERGE_FIRST_SEGMENT_ID, indexInfo()); // Delta: [12, 18), [50, 60) writeSegment(DELTA_MERGE_FIRST_SEGMENT_ID, 6, /* at */ 12); @@ -1189,6 +1182,7 @@ try ingestDTFileIntoDelta(DELTA_MERGE_FIRST_SEGMENT_ID, 5, /* at */ 0, /* clear */ false); flushSegmentCache(DELTA_MERGE_FIRST_SEGMENT_ID); mergeSegmentDelta(DELTA_MERGE_FIRST_SEGMENT_ID); + ensureSegmentStableIndex(DELTA_MERGE_FIRST_SEGMENT_ID, indexInfo()); writeSegment(DELTA_MERGE_FIRST_SEGMENT_ID, 10, /* at */ 20); @@ -1259,11 +1253,7 @@ class VectorIndexSegmentOnS3Test auto cols = DMTestEnv::getDefaultColumns(); auto vec_cd = cdVec(); - vec_cd.vector_index = std::make_shared(TiDB::VectorIndexDefinition{ - .kind = tipb::VectorIndexKind::HNSW, - .dimension = 1, - .distance_metric = tipb::VectorDistanceMetric::L2, - }); + vec_cd.vector_index = std::make_shared(index_info); cols->emplace_back(vec_cd); setColumns(cols); @@ -1346,7 +1336,7 @@ class VectorIndexSegmentOnS3Test { auto * file_cache = FileCache::instance(); auto file_segments = file_cache->getAll(); - for (const auto & file_seg : file_cache->getAll()) + for (const auto & file_seg : file_segments) file_cache->remove(file_cache->toS3Key(file_seg->getLocalFileName()), true); RUNTIME_CHECK(file_cache->getAll().empty()); @@ -1359,11 +1349,47 @@ class VectorIndexSegmentOnS3Test block.insert(colVecFloat32("[0, 100)", vec_column_name, vec_column_id)); wn_segment->write(*dm_context, std::move(block), true); wn_segment = wn_segment->mergeDelta(*dm_context, tableColumns()); + wn_segment = buildIndex(dm_context, wn_segment); + RUNTIME_CHECK(wn_segment != nullptr); // Let's just make sure we are later indeed reading from S3 RUNTIME_CHECK(wn_segment->stable->getDMFiles()[0]->path().rfind("s3://") == 0); } + SegmentPtr buildIndex(DMContextPtr dm_context, SegmentPtr segment) + { + auto * file_cache = FileCache::instance(); + RUNTIME_CHECK(file_cache != nullptr); + RUNTIME_CHECK(file_cache->getAll().empty()); + + auto dm_files = segment->getStable()->getDMFiles(); + auto build_info = DMFileIndexWriter::getLocalIndexBuildInfo(indexInfo(index_info), dm_files); + + // Build index + DMFileIndexWriter iw(DMFileIndexWriter::Options{ + .path_pool = storage_path_pool, + .file_provider = dm_context->db_context.getFileProvider(), + .write_limiter = dm_context->getWriteLimiter(), + .disagg_ctx = dm_context->db_context.getSharedContextDisagg(), + .index_infos = build_info.indexes_to_build, + .dm_files = dm_files, + .db_context = dm_context->db_context, + .is_common_handle = dm_context->is_common_handle, + .rowkey_column_size = dm_context->rowkey_column_size, + }); + auto new_dmfiles = iw.build(); + + RUNTIME_CHECK(file_cache->getAll().size() == 2); + SegmentPtr new_segment; + { + auto lock = segment->mustGetUpdateLock(); + new_segment = segment->replaceStableMetaVersion(lock, *dm_context, new_dmfiles); + } + // remove all file cache to make sure we are reading from S3 + removeAllFileCache(); + return new_segment; + } + BlockInputStreamPtr computeNodeTableScan() { return createComputeNodeStream(wn_segment, {cdPK(), cdVec()}, nullptr); @@ -1426,6 +1452,12 @@ class VectorIndexSegmentOnS3Test // MemoryTrackerPtr memory_tracker; MemTrackerWrapper dummy_mem_tracker = MemTrackerWrapper(0, root_of_query_mem_trackers.get()); + + const TiDB::VectorIndexDefinition index_info = { + .kind = tipb::VectorIndexKind::HNSW, + .dimension = 1, + .distance_metric = tipb::VectorDistanceMetric::L2, + }; }; TEST_F(VectorIndexSegmentOnS3Test, FileCacheNotEnabled) @@ -1910,5 +1942,4 @@ try } CATCH - } // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index_utils.h b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index_utils.h new file mode 100644 index 00000000000..08ffe3ec24d --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index_utils.h @@ -0,0 +1,88 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB::DM::tests +{ + +class VectorIndexTestUtils +{ +public: + const ColumnID vec_column_id = 100; + const String vec_column_name = "vec"; + + /// Create a column with values like [1], [2], [3], ... + /// Each value is a VectorFloat32 with exactly one dimension. + static ColumnWithTypeAndName colInt64(std::string_view sequence, const String & name = "", Int64 column_id = 0) + { + auto data = genSequence(sequence); + return ::DB::tests::createColumn(data, name, column_id); + } + + static ColumnWithTypeAndName colVecFloat32(std::string_view sequence, const String & name = "", Int64 column_id = 0) + { + auto data = genSequence(sequence); + std::vector data_in_array; + for (auto & v : data) + { + Array vec; + vec.push_back(static_cast(v)); + data_in_array.push_back(vec); + } + return ::DB::tests::createVecFloat32Column(data_in_array, name, column_id); + } + + static String encodeVectorFloat32(const std::vector & vec) + { + WriteBufferFromOwnString wb; + Array arr; + for (const auto & v : vec) + arr.push_back(static_cast(v)); + EncodeVectorFloat32(arr, wb); + return wb.str(); + } + + ColumnDefine cdVec() + { + // When used in read, no need to assign vector_index. + return ColumnDefine(vec_column_id, vec_column_name, ::DB::tests::typeFromString("Array(Float32)")); + } + + IndexInfosPtr indexInfo( + TiDB::VectorIndexDefinition definition = TiDB::VectorIndexDefinition{ + .kind = tipb::VectorIndexKind::HNSW, + .dimension = 1, + .distance_metric = tipb::VectorDistanceMetric::L2, + }) + { + const IndexInfos index_infos = IndexInfos{ + IndexInfo{ + .type = IndexType::Vector, + .column_id = vec_column_id, + .index_definition = std::make_shared(definition), + }, + }; + return std::make_shared(index_infos); + } +}; + +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_local_indexer_scheduler.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_local_indexer_scheduler.cpp new file mode 100644 index 00000000000..e3ef3cdb2d6 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_local_indexer_scheduler.cpp @@ -0,0 +1,589 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include +#include +#include + +namespace DB::DM::tests +{ + +class LocalIndexerSchedulerTest : public ::testing::Test +{ +protected: + void pushResult(String result) + { + std::unique_lock lock(results_mu); + results.push_back(result); + } + + std::mutex results_mu; + std::vector results; +}; + + +TEST_F(LocalIndexerSchedulerTest, StartScheduler) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 5, + .auto_start = false, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [this]() { pushResult("foo"); }, + }); + + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + ASSERT_EQ(results.size(), 0); + + scheduler.reset(); + ASSERT_EQ(results.size(), 0); + + scheduler = LocalIndexerScheduler::create({ + .pool_size = 5, + .auto_start = false, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [this]() { pushResult("bar"); }, + }); + + scheduler->start(); + scheduler->waitForFinish(); + + ASSERT_EQ(1, results.size()); + ASSERT_STREQ("bar", results[0].c_str()); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, KeyspaceFair) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 1, + .auto_start = false, + }); + + scheduler->pushTask({ + .keyspace_id = 2, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks2_t1"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 2, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t2"); }, + }); + scheduler->pushTask({ + .keyspace_id = 3, + .table_id = 3, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks3_t3"); }, + }); + scheduler->pushTask({ + .keyspace_id = 2, + .table_id = 4, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks2_t4"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t1"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 3, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t3"); }, + }); + + scheduler->start(); + scheduler->waitForFinish(); + + // Scheduler is scheduled by KeyspaceID asc order and TableID asc order. + ASSERT_EQ(results.size(), 6); + ASSERT_STREQ(results[0].c_str(), "ks1_t1"); + ASSERT_STREQ(results[1].c_str(), "ks2_t1"); + ASSERT_STREQ(results[2].c_str(), "ks3_t3"); + ASSERT_STREQ(results[3].c_str(), "ks1_t2"); + ASSERT_STREQ(results[4].c_str(), "ks2_t4"); + ASSERT_STREQ(results[5].c_str(), "ks1_t3"); + + results.clear(); + + scheduler->pushTask({ + .keyspace_id = 2, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks2_t1"); }, + }); + + scheduler->waitForFinish(); + + ASSERT_EQ(results.size(), 1); + ASSERT_STREQ(results[0].c_str(), "ks2_t1"); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, TableFair) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 1, + .auto_start = false, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 3, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t3_#1"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t1_#1"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 3, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t3_#2"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 2, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks1_t2_#1"); }, + }); + scheduler->pushTask({ + .keyspace_id = 2, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("ks2_t1_#1"); }, + }); + + scheduler->start(); + scheduler->waitForFinish(); + + // Scheduler is scheduled by KeyspaceID asc order and TableID asc order. + ASSERT_EQ(results.size(), 5); + ASSERT_STREQ(results[0].c_str(), "ks1_t1_#1"); + ASSERT_STREQ(results[1].c_str(), "ks2_t1_#1"); + ASSERT_STREQ(results[2].c_str(), "ks1_t2_#1"); + ASSERT_STREQ(results[3].c_str(), "ks1_t3_#1"); + ASSERT_STREQ(results[4].c_str(), "ks1_t3_#2"); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, TaskExceedMemoryLimit) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 10, + .memory_limit = 2, + .auto_start = false, + }); + + ASSERT_THROW( + { + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 100, + .workload = [&]() { pushResult("foo"); }, + }); + }, + DB::Exception); + ASSERT_NO_THROW({ + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("bar"); }, + }); + }); + + scheduler->start(); + scheduler->waitForFinish(); + + ASSERT_EQ(results.size(), 1); + ASSERT_STREQ(results[0].c_str(), "bar"); + + results.clear(); + + scheduler = LocalIndexerScheduler::create({ + .pool_size = 10, + .memory_limit = 0, + }); + + ASSERT_NO_THROW({ + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 100, + .workload = [&]() { pushResult("foo"); }, + }); + }); + ASSERT_NO_THROW({ + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("bar"); }, + }); + }); + + scheduler->start(); + scheduler->waitForFinish(); + + ASSERT_EQ(results.size(), 2); + ASSERT_STREQ(results[0].c_str(), "foo"); + ASSERT_STREQ(results[1].c_str(), "bar"); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, MemoryLimit) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 10, + .memory_limit = 2, + .auto_start = false, + }); + + auto task_1_is_started = std::make_shared>(); + auto task_2_is_started = std::make_shared>(); + auto task_3_is_started = std::make_shared>(); + + auto task_1_wait = std::make_shared>(); + auto task_2_wait = std::make_shared>(); + auto task_3_wait = std::make_shared>(); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 1, + .workload = + [=]() { + task_1_is_started->set_value(); + task_1_wait->get_future().wait(); + }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 1, + .workload = + [=]() { + task_2_is_started->set_value(); + task_2_wait->get_future().wait(); + }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 1, + .workload = + [=]() { + task_3_is_started->set_value(); + task_3_wait->get_future().wait(); + }, + }); + + scheduler->start(); + + task_1_is_started->get_future().wait(); + task_2_is_started->get_future().wait(); + + auto task_3_is_started_future = task_3_is_started->get_future(); + + // We should fail to got task 3 start running, because current memory limit is reached + ASSERT_EQ(task_3_is_started_future.wait_for(std::chrono::milliseconds(500)), std::future_status::timeout); + + task_1_wait->set_value(); + + task_3_is_started_future.wait(); + + task_2_wait->set_value(); + task_3_wait->set_value(); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, ShutdownWithPendingTasks) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 1, + .auto_start = false, + }); + + auto task_1_is_started = std::make_shared>(); + auto task_1_wait = std::make_shared>(); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = + [=]() { + task_1_is_started->set_value(); + task_1_wait->get_future().wait(); + }, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = + [=]() { + // Should not enter here. + ASSERT_TRUE(false); + }, + }); + + scheduler->start(); + + // Ensure task 1 is running + task_1_is_started->get_future().wait(); + + // Shutdown the scheduler. + auto shutdown_th = std::async([&]() { scheduler.reset(); }); + + // The shutdown should be waiting for task 1 to finish + ASSERT_EQ(shutdown_th.wait_for(std::chrono::milliseconds(500)), std::future_status::timeout); + + // After task 1 finished, the scheduler shutdown should be ok. + task_1_wait->set_value(); + shutdown_th.wait(); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, WorkloadException) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 1, + .auto_start = false, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { throw DB::Exception("foo"); }, + }); + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {}, + .request_memory = 0, + .workload = [&]() { pushResult("bar"); }, + }); + + scheduler->start(); + scheduler->waitForFinish(); + + ASSERT_EQ(results.size(), 1); + ASSERT_STREQ(results[0].c_str(), "bar"); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, DMFileIsUsing) +try +{ + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 4, + .auto_start = false, + }); + + auto task_1_is_started = std::make_shared>(); + auto task_2_is_started = std::make_shared>(); + auto task_3_is_started = std::make_shared>(); + + auto task_1_wait = std::make_shared>(); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {1}, + .request_memory = 0, + .workload = + [&]() { + task_1_is_started->set_value(); + task_1_wait->get_future().wait(); + }, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {1, 2}, + .request_memory = 0, + .workload = [&]() { task_2_is_started->set_value(); }, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {3}, + .request_memory = 0, + .workload = [&]() { task_3_is_started->set_value(); }, + }); + + scheduler->start(); + + task_1_is_started->get_future().wait(); + + auto task_2_is_started_future = task_2_is_started->get_future(); + // We should fail to got task 2 start running, because current dmfile is using + ASSERT_EQ(task_2_is_started_future.wait_for(std::chrono::milliseconds(500)), std::future_status::timeout); + // Task 3 is not using the dmfile, so it should run + task_3_is_started->get_future().wait(); + + // After task 1 is finished, task 2 should run + task_1_wait->set_value(); + task_2_is_started_future.wait(); + + scheduler->waitForFinish(); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, DMFileFromDifferentTable) +try +{ + // When DMFiles come from different table, should not block + + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 4, + .auto_start = false, + }); + + auto task_1_is_started = std::make_shared>(); + auto task_2_is_started = std::make_shared>(); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {1}, + .request_memory = 0, + .workload = [&]() { task_1_is_started->set_value(); }, + }); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 2, + .dmfile_ids = {1, 2}, + .request_memory = 0, + .workload = [&]() { task_2_is_started->set_value(); }, + }); + + scheduler->start(); + + task_1_is_started->get_future().wait(); + task_2_is_started->get_future().wait(); + + scheduler->waitForFinish(); +} +CATCH + + +TEST_F(LocalIndexerSchedulerTest, DMFileFromDifferentKeyspace) +try +{ + // When DMFiles come from different keyspace, should not block + + auto scheduler = LocalIndexerScheduler::create({ + .pool_size = 4, + .auto_start = false, + }); + + auto task_1_is_started = std::make_shared>(); + auto task_2_is_started = std::make_shared>(); + + scheduler->pushTask({ + .keyspace_id = 1, + .table_id = 1, + .dmfile_ids = {1}, + .request_memory = 0, + .workload = [&]() { task_1_is_started->set_value(); }, + }); + + scheduler->pushTask({ + .keyspace_id = 2, + .table_id = 1, + .dmfile_ids = {1}, + .request_memory = 0, + .workload = [&]() { task_2_is_started->set_value(); }, + }); + + scheduler->start(); + + task_1_is_started->get_future().wait(); + task_2_is_started->get_future().wait(); + + scheduler->waitForFinish(); +} +CATCH + +} // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp index 6a112d6f9e1..f2fa9830610 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -737,6 +738,41 @@ bool SegmentTestBasic::replaceSegmentStableData(PageIdU64 segment_id, const DMFi return success; } +bool SegmentTestBasic::ensureSegmentStableIndex(PageIdU64 segment_id, IndexInfosPtr local_index_infos) +{ + LOG_INFO(logger_op, "EnsureSegmentStableIndex, segment_id={}", segment_id); + + RUNTIME_CHECK(segments.find(segment_id) != segments.end()); + + bool success = false; + auto segment = segments[segment_id]; + auto dm_files = segment->getStable()->getDMFiles(); + auto build_info = DMFileIndexWriter::getLocalIndexBuildInfo(local_index_infos, dm_files); + + // Build index + DMFileIndexWriter iw(DMFileIndexWriter::Options{ + .path_pool = storage_path_pool, + .file_provider = dm_context->db_context.getFileProvider(), + .write_limiter = dm_context->getWriteLimiter(), + .disagg_ctx = dm_context->db_context.getSharedContextDisagg(), + .index_infos = build_info.indexes_to_build, + .dm_files = dm_files, + .db_context = dm_context->db_context, + .is_common_handle = dm_context->is_common_handle, + .rowkey_column_size = dm_context->rowkey_column_size, + }); + auto new_dmfiles = iw.build(); + RUNTIME_CHECK(new_dmfiles.size() == 1); + + LOG_INFO(logger_op, "EnsureSegmentStableIndex, build index done, segment_id={}", segment_id); + + // Replace stable data + success = replaceSegmentStableData(segment_id, new_dmfiles[0]); + + operation_statistics["ensureStableIndex"]++; + return success; +} + bool SegmentTestBasic::areSegmentsSharingStable(const std::vector & segments_id) const { RUNTIME_CHECK(segments_id.size() >= 2); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h index 44cdecc0948..d0cf689a7e1 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h @@ -101,6 +101,11 @@ class SegmentTestBasic : public DB::base::TiFlashStorageTestBasic */ bool replaceSegmentStableData(PageIdU64 segment_id, const DMFilePtr & file); + /** + * Returns whether segment stable index is created. + */ + bool ensureSegmentStableIndex(PageIdU64 segment_id, IndexInfosPtr local_index_infos); + Block prepareWriteBlock(Int64 start_key, Int64 end_key, bool is_deleted = false); Block prepareWriteBlockInSegmentRange( PageIdU64 segment_id, diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h index ebe2db01a17..e7e31757222 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_util.h @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#pragma once + #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp index 14da643e73b..57543880c15 100644 --- a/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp @@ -59,7 +59,7 @@ DTWorkload::DTWorkload( context->initializeGlobalPageIdAllocator(); context->initializeGlobalStoragePoolIfNeed(context->getPathPool()); Stopwatch sw; - store = std::make_unique( + store = DeltaMergeStore::createUnique( *context, true, table_info->db_name, @@ -71,6 +71,7 @@ DTWorkload::DTWorkload( table_info->handle, table_info->is_common_handle, table_info->rowkey_column_indexes.size(), + nullptr, DeltaMergeStore::Settings()); stat.init_ms = sw.elapsedMilliseconds(); LOG_INFO(log, "Init store {} ms", stat.init_ms); diff --git a/dbms/src/Storages/FormatVersion.h b/dbms/src/Storages/FormatVersion.h index d4761ae200e..f42b8cac711 100644 --- a/dbms/src/Storages/FormatVersion.h +++ b/dbms/src/Storages/FormatVersion.h @@ -160,7 +160,7 @@ inline static const StorageFormatVersion STORAGE_FORMAT_V101 = StorageFormatVers .identifier = 101, }; -inline StorageFormatVersion STORAGE_FORMAT_CURRENT = STORAGE_FORMAT_V5; +inline StorageFormatVersion STORAGE_FORMAT_CURRENT = STORAGE_FORMAT_V6; inline const StorageFormatVersion & toStorageFormat(UInt64 setting) { diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index f4f7b0a0966..34f032ce63b 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -47,6 +47,7 @@ #include #include #include +#include #include #include #include @@ -62,7 +63,6 @@ #include #include -#include namespace DB { @@ -1957,6 +1957,40 @@ SortDescription StorageDeltaMerge::getPrimarySortDescription() const return desc; } +IndexInfosPtr extractLocalIndexInfos(const TiDB::TableInfo & table_info) +{ + IndexInfosPtr index_infos = std::make_shared(); + index_infos->reserve(table_info.columns.size()); + for (const auto & col : table_info.columns) + { + // TODO: support more index type + if (col.vector_index) + { + // Vector Index requires a specific storage format to work. + if ((STORAGE_FORMAT_CURRENT.identifier > 0 && STORAGE_FORMAT_CURRENT.identifier < 6) + || STORAGE_FORMAT_CURRENT.identifier == 100) + { + LOG_ERROR( + Logger::get(), + "The current storage format is {}, which does not support building vector index. TiFlash will " + "write data without vector index.", + STORAGE_FORMAT_CURRENT.identifier); + return {}; + } + + index_infos->emplace_back(IndexInfo{ + .type = IndexType::Vector, + .column_id = col.id, + .column_name = col.name, + .index_definition = col.vector_index, + }); + } + } + + index_infos->shrink_to_fit(); + return index_infos; +} + DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore(ThreadPool * thread_pool) { if (storeInited()) @@ -1966,7 +2000,8 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore(ThreadPool * thread std::lock_guard lock(store_mutex); if (_store == nullptr) { - _store = std::make_shared( + auto index_infos = extractLocalIndexInfos(tidb_table_info); + _store = DeltaMergeStore::create( global_context, data_path_contains_database_name, table_column_info->db_name, @@ -1978,6 +2013,7 @@ DeltaMergeStorePtr & StorageDeltaMerge::getAndMaybeInitStore(ThreadPool * thread std::move(table_column_info->handle_column_define), is_common_handle, rowkey_column_size, + std::move(index_infos), DeltaMergeStore::Settings(), thread_pool); table_column_info.reset(nullptr); diff --git a/dbms/src/Storages/System/StorageSystemDTLocalIndexes.cpp b/dbms/src/Storages/System/StorageSystemDTLocalIndexes.cpp new file mode 100644 index 00000000000..a133182c7e1 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemDTLocalIndexes.cpp @@ -0,0 +1,138 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +StorageSystemDTLocalIndexes::StorageSystemDTLocalIndexes(const std::string & name_) + : name(name_) +{ + setColumns(ColumnsDescription({ + {"database", std::make_shared()}, + {"table", std::make_shared()}, + + {"tidb_database", std::make_shared()}, + {"tidb_table", std::make_shared()}, + {"keyspace_id", std::make_shared(std::make_shared())}, + {"table_id", std::make_shared()}, + {"belonging_table_id", std::make_shared()}, + + {"column_name", std::make_shared()}, + {"column_id", std::make_shared()}, + {"index_kind", std::make_shared()}, + + {"rows_stable_indexed", std::make_shared()}, // Total rows + {"rows_stable_not_indexed", std::make_shared()}, // Total rows + {"rows_delta_indexed", std::make_shared()}, // Total rows + {"rows_delta_not_indexed", std::make_shared()}, // Total rows + })); +} + +BlockInputStreams StorageSystemDTLocalIndexes::read( + const Names & column_names, + const SelectQueryInfo &, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + check(column_names); + processed_stage = QueryProcessingStage::FetchColumns; + + MutableColumns res_columns = getSampleBlock().cloneEmptyColumns(); + + SchemaNameMapper mapper; + + auto databases = context.getDatabases(); + for (const auto & d : databases) + { + String database_name = d.first; + const auto & database = d.second; + const DatabaseTiFlash * db_tiflash = typeid_cast(database.get()); + + auto it = database->getIterator(context); + for (; it->isValid(); it->next()) + { + const auto & table_name = it->name(); + auto & storage = it->table(); + if (storage->getName() != MutableSupport::delta_tree_storage_name) + continue; + + auto dm_storage = std::dynamic_pointer_cast(storage); + const auto & table_info = dm_storage->getTableInfo(); + auto table_id = table_info.id; + auto store = dm_storage->getStoreIfInited(); + if (!store) + continue; + + if (dm_storage->isTombstone()) + continue; + + auto index_stats = store->getLocalIndexStats(); + for (auto & stat : index_stats) + { + size_t j = 0; + res_columns[j++]->insert(database_name); + res_columns[j++]->insert(table_name); + + String tidb_db_name; + KeyspaceID keyspace_id = NullspaceID; + if (db_tiflash) + { + tidb_db_name = db_tiflash->getDatabaseInfo().name; + keyspace_id = db_tiflash->getDatabaseInfo().keyspace_id; + } + res_columns[j++]->insert(tidb_db_name); + String tidb_table_name = table_info.name; + res_columns[j++]->insert(tidb_table_name); + if (keyspace_id == NullspaceID) + res_columns[j++]->insert(Field()); + else + res_columns[j++]->insert(static_cast(keyspace_id)); + res_columns[j++]->insert(table_id); + res_columns[j++]->insert(table_info.belonging_table_id); + + res_columns[j++]->insert(stat.column_name); + res_columns[j++]->insert(stat.column_id); + res_columns[j++]->insert(stat.index_kind); + + res_columns[j++]->insert(stat.rows_stable_indexed); + res_columns[j++]->insert(stat.rows_stable_not_indexed); + res_columns[j++]->insert(stat.rows_delta_indexed); + res_columns[j++]->insert(stat.rows_delta_not_indexed); + } + } + } + + return BlockInputStreams( + 1, + std::make_shared(getSampleBlock().cloneWithColumns(std::move(res_columns)))); +} + + +} // namespace DB diff --git a/dbms/src/Storages/System/StorageSystemDTLocalIndexes.h b/dbms/src/Storages/System/StorageSystemDTLocalIndexes.h new file mode 100644 index 00000000000..d53e0fe3262 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemDTLocalIndexes.h @@ -0,0 +1,50 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +#include + + +namespace DB +{ +class Context; + + +class StorageSystemDTLocalIndexes + : public ext::SharedPtrHelper + , public IStorage +{ +public: + std::string getName() const override { return "SystemDTLocalIndexes"; } + std::string getTableName() const override { return name; } + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + const std::string name; + +protected: + StorageSystemDTLocalIndexes(const std::string & name_); +}; + +} // namespace DB diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index 0092d984d49..0502501a891 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -40,6 +40,7 @@ StorageSystemDTSegments::StorageSystemDTSegments(const std::string & name_) {"tidb_table", std::make_shared()}, {"keyspace_id", std::make_shared(std::make_shared())}, {"table_id", std::make_shared()}, + {"belonging_table_id", std::make_shared()}, {"is_tombstone", std::make_shared()}, {"segment_id", std::make_shared()}, @@ -131,6 +132,7 @@ BlockInputStreams StorageSystemDTSegments::read( else res_columns[j++]->insert(static_cast(keyspace_id)); res_columns[j++]->insert(table_id); + res_columns[j++]->insert(table_info.belonging_table_id); res_columns[j++]->insert(dm_storage->getTombstone()); res_columns[j++]->insert(stat.segment_id); diff --git a/dbms/src/Storages/System/StorageSystemDTTables.cpp b/dbms/src/Storages/System/StorageSystemDTTables.cpp index 2a0d45957a1..33c8813128d 100644 --- a/dbms/src/Storages/System/StorageSystemDTTables.cpp +++ b/dbms/src/Storages/System/StorageSystemDTTables.cpp @@ -41,6 +41,7 @@ StorageSystemDTTables::StorageSystemDTTables(const std::string & name_) {"tidb_table", std::make_shared()}, {"keyspace_id", std::make_shared(std::make_shared())}, {"table_id", std::make_shared()}, + {"belonging_table_id", std::make_shared()}, {"is_tombstone", std::make_shared()}, {"segment_count", std::make_shared()}, @@ -163,6 +164,7 @@ BlockInputStreams StorageSystemDTTables::read( else res_columns[j++]->insert(static_cast(keyspace_id)); res_columns[j++]->insert(table_id); + res_columns[j++]->insert(table_info.belonging_table_id); res_columns[j++]->insert(dm_storage->getTombstone()); res_columns[j++]->insert(stat.segment_count); diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index d20bbe68a79..6be10c8b564 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -42,6 +43,7 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("databases", StorageSystemDatabases::create("databases")); system_database.attachTable("dt_tables", StorageSystemDTTables::create("dt_tables")); system_database.attachTable("dt_segments", StorageSystemDTSegments::create("dt_segments")); + system_database.attachTable("dt_local_indexes", StorageSystemDTLocalIndexes::create("dt_local_indexes")); system_database.attachTable("tables", StorageSystemTables::create("tables")); system_database.attachTable("columns", StorageSystemColumns::create("columns")); system_database.attachTable("functions", StorageSystemFunctions::create("functions")); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 78f6692338d..81c0e3a72f9 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -117,6 +117,8 @@ void TiFlashTestEnv::addGlobalContext( KeyManagerPtr key_manager = std::make_shared(false); global_context->initializeFileProvider(key_manager, false); + global_context->initializeGlobalLocalIndexerScheduler(1, 0); + // initialize background & blockable background thread pool global_context->setSettings(settings_); Settings & settings = global_context->getSettingsRef(); diff --git a/dbms/src/TiDB/Schema/TiDB.cpp b/dbms/src/TiDB/Schema/TiDB.cpp index cc6058e7139..fd6deec1256 100644 --- a/dbms/src/TiDB/Schema/TiDB.cpp +++ b/dbms/src/TiDB/Schema/TiDB.cpp @@ -224,6 +224,8 @@ Field ColumnInfo::defaultValueToField() const return getYearValue(value.convert()); case TypeSet: TRY_CATCH_DEFAULT_VALUE_TO_FIELD({ return getSetValue(value.convert()); }); + case TypeTiDBVectorFloat32: + return genVectorFloat32Empty(); default: throw Exception("Have not processed type: " + std::to_string(tp)); } @@ -1274,6 +1276,12 @@ String genJsonNull() return null; } +String genVectorFloat32Empty() +{ + String v(4, '\0'); // Length=0 vector + return v; +} + tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci) { tipb::FieldType ret; diff --git a/dbms/src/TiDB/Schema/TiDB.h b/dbms/src/TiDB/Schema/TiDB.h index 5d30b950b92..da5e50f0658 100644 --- a/dbms/src/TiDB/Schema/TiDB.h +++ b/dbms/src/TiDB/Schema/TiDB.h @@ -420,6 +420,8 @@ using DBInfoPtr = std::shared_ptr; String genJsonNull(); +String genVectorFloat32Empty(); + tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci); ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); ColumnInfo toTiDBColumnInfo(const tipb::ColumnInfo & tipb_column_info);