From fc33aff202b04c2ff65e2af458db80aaf9b1dd69 Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 6 Sep 2024 21:22:23 +0800 Subject: [PATCH] Storage: Support drop vector index defined on ColumnInfo when column is dropped (#283) --- dbms/src/Common/FailPoint.cpp | 1 + .../Storages/DeltaMerge/DeltaMergeStore.cpp | 31 +- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 27 +- .../DeltaMerge/File/DMFileIndexWriter.cpp | 3 + .../Storages/DeltaMerge/Index/IndexInfo.cpp | 274 ++++++++++++------ ...test_dm_delta_merge_store_vector_index.cpp | 126 +++++++- .../tests/gtest_local_index_info.cpp | 122 ++++++++ dbms/src/TiDB/Schema/TiDB.cpp | 2 +- 8 files changed, 470 insertions(+), 116 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index ae5eeaf0d75..7446eeb8812 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -71,6 +71,7 @@ namespace DB M(force_remote_read_for_batch_cop_once) \ M(exception_new_dynamic_thread) \ M(force_wait_index_timeout) \ + M(force_not_support_vector_index) \ M(sync_schema_request_failure) #define APPLY_FOR_FAILPOINTS(M) \ diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index df122ee2489..f95980967b6 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -2020,25 +2020,32 @@ void DeltaMergeStore::applySchemaChanges(TiDB::TableInfo & table_info) original_table_columns.swap(new_original_table_columns); store_columns.swap(new_store_columns); - // Get a snapshot on the local_index_infos to check whether any new index is created - LocalIndexInfosSnapshot local_index_infos_snap = getLocalIndexInfosSnapshot(); - std::atomic_store(&original_table_header, std::make_shared(toEmptyBlock(original_table_columns))); // release the lock because `checkAllSegmentsLocalIndex` will try to acquire the lock // and generate tasks on segments lock.unlock(); - auto new_local_index_infos = generateLocalIndexInfos(local_index_infos_snap, table_info, log); - if (new_local_index_infos) + applyLocalIndexChange(table_info); +} + +void DeltaMergeStore::applyLocalIndexChange(const TiDB::TableInfo & new_table_info) +{ + // Get a snapshot on the local_index_infos to check whether any new index is created + auto new_local_index_infos = generateLocalIndexInfos(getLocalIndexInfosSnapshot(), new_table_info, log); + + // no index is created or dropped + if (!new_local_index_infos) + return; + { - { - // new index created, update the info in-memory thread safety between `getLocalIndexInfosSnapshot` - std::unique_lock index_write_lock(mtx_local_index_infos); - local_index_infos.swap(new_local_index_infos); - } - checkAllSegmentsLocalIndex(); - } // else no new index is created + // new index created, update the info in-memory thread safety between `getLocalIndexInfosSnapshot` + std::unique_lock index_write_lock(mtx_local_index_infos); + local_index_infos.swap(new_local_index_infos); + } + + // generate async tasks for building local index for all segments + checkAllSegmentsLocalIndex(); } SortDescription DeltaMergeStore::getPrimarySortDescription() const diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index b71d913cb8a..32f332d4558 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -582,6 +582,18 @@ class DeltaMergeStore bool keep_order, const PushDownFilterPtr & filter); + // Get a snap of local_index_infos for checking. + // Note that this is just a shallow copy of `local_index_infos`, do not + // modify the local indexes inside the snapshot. + LocalIndexInfosSnapshot getLocalIndexInfosSnapshot() const + { + std::shared_lock index_read_lock(mtx_local_index_infos); + if (!local_index_infos || local_index_infos->empty()) + return nullptr; + // only make a shallow copy on the shared_ptr is OK + return local_index_infos; + } + public: /// Methods mainly used by region split. @@ -863,18 +875,6 @@ class DeltaMergeStore const SegmentPtr & segment, const DMFiles & new_dm_files); - // Get a snap of local_index_infos for checking. - // Note that this is just a shallow copy of `local_index_infos`, do not - // modify the local indexes inside the snapshot. - LocalIndexInfosSnapshot getLocalIndexInfosSnapshot() const - { - std::shared_lock index_read_lock(mtx_local_index_infos); - if (!local_index_infos || local_index_infos->empty()) - return nullptr; - // only make a shallow copy on the shared_ptr is OK - return local_index_infos; - } - /** * Check whether there are new local indexes should be built for all segments. */ @@ -894,6 +894,9 @@ class DeltaMergeStore #else public: #endif + + void applyLocalIndexChange(const TiDB::TableInfo & new_table_info); + /** * Wait until the segment has stable index. * If the index is ready or no need to build, it will return immediately. diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp index 33c342e0b17..da58ca0696e 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileIndexWriter.cpp @@ -43,6 +43,9 @@ DMFileIndexWriter::LocalIndexBuildInfo DMFileIndexWriter::getLocalIndexBuildInfo assert(index_infos != nullptr); static constexpr double VECTOR_INDEX_SIZE_FACTOR = 1.2; + // TODO(vector-index): Now we only generate the build info when new index is added. + // The built indexes will be dropped (lazily) after the segment instance is updated. + // We can support dropping the vector index more quickly later. LocalIndexBuildInfo build; build.indexes_to_build = std::make_shared(); build.file_ids.reserve(dm_files.size()); diff --git a/dbms/src/Storages/DeltaMerge/Index/IndexInfo.cpp b/dbms/src/Storages/DeltaMerge/Index/IndexInfo.cpp index 8eafd257ba4..6981e516f05 100644 --- a/dbms/src/Storages/DeltaMerge/Index/IndexInfo.cpp +++ b/dbms/src/Storages/DeltaMerge/Index/IndexInfo.cpp @@ -12,16 +12,56 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include #include +#include -#include - +namespace DB::FailPoints +{ +extern const char force_not_support_vector_index[]; +} // namespace DB::FailPoints namespace DB::DM { +struct ComplexIndexID +{ + IndexID index_id; + ColumnID column_id; +}; + +bool operator==(const ComplexIndexID & lhs, const ComplexIndexID & rhs) +{ + if (lhs.index_id != rhs.index_id) + return false; + return lhs.column_id == rhs.column_id; +} + +struct ComplexIndexIDHasher +{ + std::size_t operator()(const ComplexIndexID & id) const + { + using boost::hash_combine; + using boost::hash_value; + + std::size_t seed = 0; + if (id.index_id > EmptyIndexID) + { + hash_combine(seed, hash_value(0x01)); + hash_combine(seed, hash_value(id.index_id)); + } + else + { + hash_combine(seed, hash_value(0x02)); + hash_combine(seed, hash_value(id.column_id)); + } + return seed; + } +}; + + bool isVectorIndexSupported(const LoggerPtr & logger) { // Vector Index requires a specific storage format to work. @@ -39,22 +79,17 @@ bool isVectorIndexSupported(const LoggerPtr & logger) return true; } -TiDB::ColumnInfo getVectorIndxColumnInfo( +ColumnID getVectorIndxColumnID( const TiDB::TableInfo & table_info, const TiDB::IndexInfo & idx_info, const LoggerPtr & logger) { - if (!idx_info.vector_index - || (idx_info.state != TiDB::StatePublic && idx_info.state != TiDB::StateWriteReorganization)) - { - return {}; - } + if (!idx_info.vector_index) + return EmptyColumnID; // Vector Index requires a specific storage format to work. - if (!isVectorIndexSupported(logger)) - { - return {}; - } + if (unlikely(!isVectorIndexSupported(logger))) + return EmptyColumnID; if (idx_info.idx_cols.size() != 1) { @@ -64,14 +99,14 @@ TiDB::ColumnInfo getVectorIndxColumnInfo( idx_info.idx_cols.size(), idx_info.id, table_info.id); - return {}; + return EmptyColumnID; } for (const auto & col : table_info.columns) { if (col.name == idx_info.idx_cols[0].name) { - return col; + return col.id; } } @@ -81,51 +116,13 @@ TiDB::ColumnInfo getVectorIndxColumnInfo( table_info.id, idx_info.id, idx_info.idx_cols[0].name); - return {}; + return EmptyColumnID; } LocalIndexInfosPtr initLocalIndexInfos(const TiDB::TableInfo & table_info, const LoggerPtr & logger) { - LocalIndexInfosPtr index_infos = std::make_shared(); - index_infos->reserve(table_info.columns.size() + table_info.index_infos.size()); - for (const auto & col : table_info.columns) - { - if (col.vector_index && isVectorIndexSupported(logger)) - { - index_infos->emplace_back(LocalIndexInfo{ - .type = IndexType::Vector, - .column_id = col.id, - .index_definition = col.vector_index, - }); - LOG_INFO(logger, "Add a new index by column comments, column_id={}, table_id={}.", col.id, table_info.id); - } - } - - for (const auto & idx : table_info.index_infos) - { - auto column = getVectorIndxColumnInfo(table_info, idx, logger); - // column.id <= 0 means we don't get the valid column ID. - if (column.id <= DB::EmptyColumnID) - { - 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 {}; - } - - LOG_INFO(logger, "Add a new index, index_id={}, table_id={}.", idx.id, table_info.id); - index_infos->emplace_back(LocalIndexInfo{ - .type = IndexType::Vector, - .index_id = idx.id, - .column_id = column.id, - .index_definition = idx.vector_index, - }); - } - - index_infos->shrink_to_fit(); - return index_infos; + // The same as generate local index infos with no existing_indexes + return generateLocalIndexInfos(nullptr, table_info, logger); } LocalIndexInfosPtr generateLocalIndexInfos( @@ -133,72 +130,173 @@ LocalIndexInfosPtr generateLocalIndexInfos( const TiDB::TableInfo & new_table_info, const LoggerPtr & logger) { - LocalIndexInfosPtr new_index_infos = std::make_shared>(); - // The first time generate index infos. - if (!existing_indexes) + LocalIndexInfosPtr new_index_infos = std::make_shared(); { - auto index_infos = initLocalIndexInfos(new_table_info, logger); - if (!index_infos || index_infos->empty()) - return nullptr; - new_index_infos = std::move(index_infos); - return new_index_infos; + // If the storage format does not support vector index, always return an empty + // index_info. Meaning we should drop all indexes + bool is_storage_format_support = isVectorIndexSupported(logger); + fiu_do_on(FailPoints::force_not_support_vector_index, { is_storage_format_support = false; }); + if (!is_storage_format_support) + return new_index_infos; + } + + // Keep a map of "indexes in existing_indexes" -> "offset in new_index_infos" + std::unordered_map original_local_index_id_map; + if (existing_indexes) + { + // Create a copy of existing indexes + for (size_t offset = 0; offset < existing_indexes->size(); ++offset) + { + const auto & index = (*existing_indexes)[offset]; + original_local_index_id_map.emplace( + ComplexIndexID{.index_id = index.index_id, .column_id = index.column_id}, + offset); + new_index_infos->emplace_back(index); + } } - new_index_infos->insert(new_index_infos->cend(), existing_indexes->begin(), existing_indexes->end()); + std::unordered_set index_ids_in_new_table; + std::vector newly_added; + std::vector newly_dropped; - std::unordered_map original_local_index_id_map; - for (size_t index = 0; index < new_index_infos->size(); ++index) + // In the serverless branch, previously we define vector index on TiDB::ColumnInfo + for (const auto & col : new_table_info.columns) { - original_local_index_id_map[new_index_infos->at(index).index_id] = index; + if (!col.vector_index) + continue; + + // We do the check at the beginning, only assert check under debug mode + // is enough + assert(isVectorIndexSupported(logger)); + + const ComplexIndexID cindex_id{.index_id = EmptyIndexID, .column_id = col.id}; + index_ids_in_new_table.emplace(cindex_id); + // already exist in `existing_indexes` + if (original_local_index_id_map.contains(cindex_id)) + continue; + // newly added + new_index_infos->emplace_back(LocalIndexInfo{ + .type = IndexType::Vector, + .index_id = EmptyIndexID, // the vector index created on ColumnInfo, use EmptyIndexID as the index_id + .column_id = col.id, + .index_definition = col.vector_index, + }); + newly_added.emplace_back(cindex_id); } - bool any_new_index_created = false; - bool any_index_removed = false; for (const auto & idx : new_table_info.index_infos) { if (!idx.vector_index) continue; - auto iter = original_local_index_id_map.find(idx.id); + const auto column_id = getVectorIndxColumnID(new_table_info, idx, logger); + if (column_id <= EmptyColumnID) + continue; + + const ComplexIndexID cindex_id{.index_id = idx.id, .column_id = column_id}; + auto iter = original_local_index_id_map.find(cindex_id); if (iter == original_local_index_id_map.end()) { if (idx.state == TiDB::StatePublic || idx.state == TiDB::StateWriteReorganization) { // create a new index - auto column = getVectorIndxColumnInfo(new_table_info, idx, logger); - LocalIndexInfo index_info{ + new_index_infos->emplace_back(LocalIndexInfo{ .type = IndexType::Vector, .index_id = idx.id, - .column_id = column.id, + .column_id = column_id, .index_definition = idx.vector_index, - }; - new_index_infos->emplace_back(std::move(index_info)); - any_new_index_created = true; - LOG_INFO(logger, "Add a new index, index_id={}, table_id={}.", idx.id, new_table_info.id); + }); + newly_added.emplace_back(cindex_id); + index_ids_in_new_table.emplace(cindex_id); } + // else the index is not public or write reorg, consider this index as not exist } else { - if (idx.state == TiDB::StateDeleteReorganization) - continue; - // remove the existing index - original_local_index_id_map.erase(iter); + if (idx.state != TiDB::StateDeleteReorganization) + index_ids_in_new_table.emplace(cindex_id); + // else exist in both `existing_indexes` and `new_table_info`, but enter "delete reorg". We consider this + // index as not exist in the `new_table_info` and drop it later } } // drop nonexistent indexes - for (auto & iter : original_local_index_id_map) + for (auto iter = original_local_index_id_map.begin(); iter != original_local_index_id_map.end(); /* empty */) { - // It means this index is create by column comments which we don't support drop index. - if (iter.first == DB::EmptyIndexID) + // the index_id exists in both `existing_indexes` and `new_table_info` + if (index_ids_in_new_table.contains(iter->first)) + { + ++iter; continue; - new_index_infos->erase(new_index_infos->begin() + iter.second); - any_index_removed = true; - LOG_INFO(logger, "Drop a index, index_id={}, table_id={}.", iter.first, new_table_info.id); + } + + // not exists in `new_table_info`, drop it + newly_dropped.emplace_back(iter->first); + new_index_infos->erase(new_index_infos->begin() + iter->second); + iter = original_local_index_id_map.erase(iter); } - if (!any_new_index_created && !any_index_removed) + if (newly_added.empty() && newly_dropped.empty()) + { + auto get_logging = [&]() -> String { + FmtBuffer buf; + buf.append("keep=["); + buf.joinStr( + original_local_index_id_map.begin(), + original_local_index_id_map.end(), + [](const auto & id, FmtBuffer & fb) { + if (id.first.index_id != EmptyIndexID) + fb.fmtAppend("index_id={}", id.first.index_id); + else + fb.fmtAppend("index_on_column_id={}", id.first.column_id); + }, + ","); + buf.append("]"); + return buf.toString(); + }; + LOG_DEBUG(logger, "Local index info does not changed, {}", get_logging()); return nullptr; + } + + auto get_changed_logging = [&]() -> String { + FmtBuffer buf; + buf.append("keep=["); + buf.joinStr( + original_local_index_id_map.begin(), + original_local_index_id_map.end(), + [](const auto & id, FmtBuffer & fb) { + if (id.first.index_id != EmptyIndexID) + fb.fmtAppend("index_id={}", id.first.index_id); + else + fb.fmtAppend("index_on_column_id={}", id.first.column_id); + }, + ","); + buf.append("] added=["); + buf.joinStr( + newly_added.begin(), + newly_added.end(), + [](const ComplexIndexID & id, FmtBuffer & fb) { + if (id.index_id != EmptyIndexID) + fb.fmtAppend("index_id={}", id.index_id); + else + fb.fmtAppend("index_on_column_id={}", id.column_id); + }, + ","); + buf.append("] dropped=["); + buf.joinStr( + newly_dropped.begin(), + newly_dropped.end(), + [](const ComplexIndexID & id, FmtBuffer & fb) { + if (id.index_id != EmptyIndexID) + fb.fmtAppend("index_id={}", id.index_id); + else + fb.fmtAppend("index_on_column_id={}", id.column_id); + }, + ","); + buf.append("]"); + return buf.toString(); + }; + LOG_INFO(logger, "Local index info generated, {}", get_changed_logging()); return new_index_infos; } 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 index 277174a5493..f9360b2830a 100644 --- 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 @@ -14,8 +14,10 @@ #include #include +#include #include #include +#include #include @@ -33,7 +35,7 @@ class DeltaMergeStoreVectorTest store = reload(); } - DeltaMergeStorePtr reload() + DeltaMergeStorePtr reload(LocalIndexInfosPtr default_local_index = nullptr) { TiFlashStorageTestBasic::reload(); auto cols = DMTestEnv::getDefaultColumns(); @@ -41,6 +43,9 @@ class DeltaMergeStoreVectorTest ColumnDefine handle_column_define = (*cols)[0]; + if (!default_local_index) + default_local_index = indexInfo(); + DeltaMergeStorePtr s = DeltaMergeStore::create( *db_context, false, @@ -54,7 +59,7 @@ class DeltaMergeStoreVectorTest handle_column_define, false, 1, - indexInfo(), + default_local_index, DeltaMergeStore::Settings()); return s; } @@ -93,7 +98,7 @@ class DeltaMergeStoreVectorTest })); } - void triggerMergeDelta() + void triggerMergeDelta() const { std::vector all_segments; { @@ -516,6 +521,18 @@ TEST_F(DeltaMergeStoreVectorTest, TestStoreRestore) try { store = reload(); + { + auto local_index_snap = store->getLocalIndexInfosSnapshot(); + ASSERT_NE(local_index_snap, nullptr); + ASSERT_EQ(local_index_snap->size(), 1); + const auto & index = (*local_index_snap)[0]; + ASSERT_EQ(index.type, IndexType::Vector); + ASSERT_EQ(index.index_id, EmptyIndexID); + ASSERT_EQ(index.column_id, vec_column_id); + ASSERT_EQ(index.index_definition->kind, tipb::VectorIndexKind::HNSW); + ASSERT_EQ(index.index_definition->dimension, 1); + ASSERT_EQ(index.index_definition->distance_metric, tipb::VectorDistanceMetric::L2); + } const size_t num_rows_write = 128; @@ -533,6 +550,18 @@ try // check stable index has built for all segments waitStableIndexReady(); + { + auto local_index_snap = store->getLocalIndexInfosSnapshot(); + ASSERT_NE(local_index_snap, nullptr); + ASSERT_EQ(local_index_snap->size(), 1); + const auto & index = (*local_index_snap)[0]; + ASSERT_EQ(index.type, IndexType::Vector); + ASSERT_EQ(index.index_id, EmptyIndexID); + ASSERT_EQ(index.column_id, vec_column_id); + ASSERT_EQ(index.index_definition->kind, tipb::VectorIndexKind::HNSW); + ASSERT_EQ(index.index_definition->dimension, 1); + ASSERT_EQ(index.index_definition->distance_metric, tipb::VectorDistanceMetric::L2); + } const auto range = RowKeyRange::newAll(store->is_common_handle, store->rowkey_column_size); @@ -567,4 +596,95 @@ try } CATCH +TEST_F(DeltaMergeStoreVectorTest, DDLAddVectorIndex) +try +{ + { + auto indexes = std::make_shared(); + store = reload(indexes); + ASSERT_EQ(store->getLocalIndexInfosSnapshot(), nullptr); + } + + const size_t num_rows_write = 128; + + // write to store before index built + write(num_rows_write); + // trigger mergeDelta for all segments + triggerMergeDelta(); + + { + // Add vecotr index + TiDB::TableInfo new_table_info_with_vector_index; + TiDB::ColumnInfo column_info; + column_info.name = VectorIndexTestUtils::vec_column_name; + column_info.id = VectorIndexTestUtils::vec_column_id; + new_table_info_with_vector_index.columns.emplace_back(column_info); + TiDB::IndexInfo index; + index.id = 2; + TiDB::IndexColumnInfo index_col_info; + index_col_info.name = VectorIndexTestUtils::vec_column_name; + index_col_info.offset = 0; + index.idx_cols.emplace_back(index_col_info); + index.vector_index = TiDB::VectorIndexDefinitionPtr(new TiDB::VectorIndexDefinition{ + .kind = tipb::VectorIndexKind::HNSW, + .dimension = 1, + .distance_metric = tipb::VectorDistanceMetric::L2, + }); + new_table_info_with_vector_index.index_infos.emplace_back(index); + // apply local index change, shuold + // - create the local index + // - generate the background tasks for building index on stable + store->applyLocalIndexChange(new_table_info_with_vector_index); + ASSERT_EQ(store->local_index_infos->size(), 1); + } + + // 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_index_id(2); + 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}})); + } + + { + // vector index is dropped + TiDB::TableInfo new_table_info_with_vector_index; + TiDB::ColumnInfo column_info; + column_info.name = VectorIndexTestUtils::vec_column_name; + column_info.id = VectorIndexTestUtils::vec_column_id; + new_table_info_with_vector_index.columns.emplace_back(column_info); + // apply local index change, shuold drop the local index + store->applyLocalIndexChange(new_table_info_with_vector_index); + ASSERT_EQ(store->local_index_infos->size(), 0); + } +} +CATCH + } // namespace DB::DM::tests diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_local_index_info.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_local_index_info.cpp index 8caf59b0eb4..33496a5f0c5 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_local_index_info.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_local_index_info.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -19,9 +20,64 @@ #include #include +namespace DB::FailPoints +{ +extern const char force_not_support_vector_index[]; +} // namespace DB::FailPoints namespace DB::DM::tests { +TEST(LocalIndexInfoTest, StorageFormatNotSupport) +try +{ + TiDB::TableInfo table_info; + { + TiDB::ColumnInfo column_info; + column_info.name = "vec"; + column_info.id = 100; + table_info.columns.emplace_back(column_info); + } + + auto logger = Logger::get(); + LocalIndexInfosPtr index_info = nullptr; + // check the same + { + auto new_index_info = generateLocalIndexInfos(index_info, table_info, logger); + ASSERT_EQ(new_index_info, nullptr); + // check again, nothing changed, return nullptr + ASSERT_EQ(nullptr, generateLocalIndexInfos(new_index_info, table_info, logger)); + + // update + index_info = new_index_info; + } + + // Add a vector index to the TableInfo. + TiDB::IndexColumnInfo default_index_col_info; + default_index_col_info.name = "vec"; + default_index_col_info.length = -1; + default_index_col_info.offset = 0; + TiDB::IndexInfo expect_idx; + { + expect_idx.id = 1; + expect_idx.idx_cols.emplace_back(default_index_col_info); + expect_idx.vector_index = TiDB::VectorIndexDefinitionPtr(new TiDB::VectorIndexDefinition{ + .kind = tipb::VectorIndexKind::HNSW, + .dimension = 1, + .distance_metric = tipb::VectorDistanceMetric::L2, + }); + table_info.index_infos.emplace_back(expect_idx); + } + + FailPointHelper::enableFailPoint(FailPoints::force_not_support_vector_index); + + // check the result when storage format not support + auto new_index_info = generateLocalIndexInfos(index_info, table_info, logger); + ASSERT_NE(new_index_info, nullptr); + // always return empty index_info, we need to drop all existing indexes + ASSERT_TRUE(new_index_info->empty()); +} +CATCH + TEST(LocalIndexInfoTest, CheckIndexChanged) try { @@ -289,4 +345,70 @@ try } CATCH +TEST(LocalIndexInfoTest, CheckIndexDropDefinedInColumnInfo) +{ + auto logger = Logger::get(); + + TiDB::TableInfo table_info; + { + // The serverless branch, vector index may directly defined + // on the ColumnInfo + TiDB::ColumnInfo column_info_v1; + column_info_v1.name = "vec1"; + column_info_v1.id = 99; + column_info_v1.vector_index = TiDB::VectorIndexDefinitionPtr(new TiDB::VectorIndexDefinition{ + .kind = tipb::VectorIndexKind::HNSW, + .dimension = 3, + .distance_metric = tipb::VectorDistanceMetric::INNER_PRODUCT, + }); + table_info.columns.emplace_back(column_info_v1); + + // A column without vector index + TiDB::ColumnInfo column_info_v2; + column_info_v2.name = "vec2"; + column_info_v2.id = 100; + table_info.columns.emplace_back(column_info_v2); + } + + LocalIndexInfosPtr index_info = nullptr; + { + // check the different with nullptr + auto new_index_info = generateLocalIndexInfos(index_info, table_info, logger); + ASSERT_NE(nullptr, new_index_info); + ASSERT_EQ(new_index_info->size(), 1); + const auto & idx0 = (*new_index_info)[0]; + ASSERT_EQ(IndexType::Vector, idx0.type); + ASSERT_EQ(EmptyIndexID, idx0.index_id); // the vector index defined on ColumnInfo + ASSERT_EQ(99, idx0.column_id); + ASSERT_NE(nullptr, idx0.index_definition); + ASSERT_EQ(tipb::VectorIndexKind::HNSW, idx0.index_definition->kind); + ASSERT_EQ(3, idx0.index_definition->dimension); + ASSERT_EQ(tipb::VectorDistanceMetric::INNER_PRODUCT, idx0.index_definition->distance_metric); + + // check again, nothing changed, return nullptr + ASSERT_EQ(nullptr, generateLocalIndexInfos(new_index_info, table_info, logger)); + + // update + index_info = new_index_info; + } + + // drop column along with index info defined in column info + table_info.columns.erase(table_info.columns.begin()); + { + // check the different with existing index_info + auto new_index_info = generateLocalIndexInfos(index_info, table_info, logger); + ASSERT_NE(nullptr, new_index_info); + // not null + ASSERT_NE(new_index_info, nullptr); + // has been dropped + ASSERT_EQ(new_index_info->size(), 0); + + // check again, nothing changed, return nullptr + ASSERT_EQ(nullptr, generateLocalIndexInfos(new_index_info, table_info, logger)); + + // update + index_info = new_index_info; + } +} + } // namespace DB::DM::tests diff --git a/dbms/src/TiDB/Schema/TiDB.cpp b/dbms/src/TiDB/Schema/TiDB.cpp index e67168d1633..ecd3b343840 100644 --- a/dbms/src/TiDB/Schema/TiDB.cpp +++ b/dbms/src/TiDB/Schema/TiDB.cpp @@ -1064,7 +1064,7 @@ try { auto index_info_json = index_arr->getObject(i); IndexInfo index_info(index_info_json); - // We only keep the "primary index" in tiflash now + // We only keep the "primary index" or "vector index" in tiflash now if (index_info.is_primary) { has_primary_index = true;