diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index fb6a4d99911..e934139b4c1 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include #include @@ -657,25 +656,6 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva Gauge, \ F(type_send, {{"type", "send_queue"}}), \ F(type_receive, {{"type", "recv_queue"}})) \ - M(tiflash_compute_request_unit, \ - "Request Unit used by tiflash compute", \ - Counter, \ - F(type_mpp, \ - {{"type", "mpp"}, \ - ComputeLabelHolder::instance().getClusterIdLabel(), \ - ComputeLabelHolder::instance().getProcessIdLabel()}), \ - F(type_cop, \ - {{"type", "cop"}, \ - ComputeLabelHolder::instance().getClusterIdLabel(), \ - ComputeLabelHolder::instance().getProcessIdLabel()}), \ - F(type_cop_stream, \ - {{"type", "cop_stream"}, \ - ComputeLabelHolder::instance().getClusterIdLabel(), \ - ComputeLabelHolder::instance().getProcessIdLabel()}), \ - F(type_batch, \ - {{"type", "batch"}, \ - ComputeLabelHolder::instance().getClusterIdLabel(), \ - ComputeLabelHolder::instance().getProcessIdLabel()})) \ M(tiflash_shared_block_schemas, \ "statistics about shared block schemas of ColumnFiles", \ Gauge, \ @@ -856,7 +836,18 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva F(type_gac_req_acquire_tokens, {"type", "gac_req_acquire_tokens"}), \ F(type_gac_req_ru_consumption_delta, {"type", "gac_req_ru_consumption_delta"}), \ F(type_gac_resp_tokens, {"type", "gac_resp_tokens"}), \ - F(type_gac_resp_capacity, {"type", "gac_resp_capacity"})) \ + F(type_gac_resp_capacity, {"type", "gac_resp_capacity"}), \ + F(type_handling_mpp_task_dispatch, {"type", "handling_mpp_task_dispatch"}), \ + F(type_handling_mpp_task_establish, {"type", "handling_mpp_task_establish"}), \ + F(type_handling_mpp_task_cancel, {"type", "handling_mpp_task_cancel"}), \ + F(type_handling_mpp_task_run, {"type", "handling_mpp_task_run"})) \ + M(tiflash_compute_request_unit, \ + "Request Unit used by tiflash compute for each resource group", \ + Counter, \ + F(type_mpp, {"type", "mpp"}), \ + F(type_cop, {"type", "cop"}), \ + F(type_cop_stream, {"type", "cop_stream"}), \ + F(type_batch, {"type", "batch"}), ) \ M(tiflash_vector_index_memory_usage, \ "Vector index memory usage", \ Gauge, \ @@ -884,7 +875,24 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva M(tiflash_read_thread_internal_us, \ "Durations of read thread internal components", \ Histogram, \ - F(type_block_queue_pop_latency, {{"type", "block_queue_pop_latency"}}, ExpBuckets{1, 2, 20})) + F(type_block_queue_pop_latency, {{"type", "block_queue_pop_latency"}}, ExpBuckets{1, 2, 20}), \ + F(type_schedule_one_round, {{"type", "schedule_one_round"}}, ExpBuckets{1, 2, 20})) \ + M(tiflash_storage_pack_compression_algorithm_count, \ + "The count of the compression algorithm used by each data part", \ + Counter, \ + F(type_constant, {"type", "constant"}), \ + F(type_constant_delta, {"type", "constant_delta"}), \ + F(type_runlength, {"type", "runlength"}), \ + F(type_for, {"type", "for"}), \ + F(type_delta_for, {"type", "delta_for"}), \ + F(type_lz4, {"type", "lz4"})) \ + M(tiflash_storage_pack_compression_bytes, \ + "The uncompression/compression bytes of lz4 and lightweight", \ + Counter, \ + F(type_lz4_compressed_bytes, {"type", "lz4_compressed_bytes"}), \ + F(type_lz4_uncompressed_bytes, {"type", "lz4_uncompressed_bytes"}), \ + F(type_lightweight_compressed_bytes, {"type", "lightweight_compressed_bytes"}), \ + F(type_lightweight_uncompressed_bytes, {"type", "lightweight_uncompressed_bytes"})) /// Buckets with boundaries [start * base^0, start * base^1, ..., start * base^(size-1)] diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeDefines.h b/dbms/src/Storages/DeltaMerge/DeltaMergeDefines.h index 98f86356d78..3c8c7256a00 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeDefines.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeDefines.h @@ -89,22 +89,11 @@ struct ColumnDefine DataTypePtr type; Field default_value; - /// Note: ColumnDefine is used in both Write path and Read path. - /// In the read path, vector_index is usually not available. Use AnnQueryInfo for - /// read related vector index information. - TiDB::VectorIndexDefinitionPtr vector_index; - - explicit ColumnDefine( - ColId id_ = 0, - String name_ = "", - DataTypePtr type_ = nullptr, - Field default_value_ = Field{}, - TiDB::VectorIndexDefinitionPtr vector_index_ = nullptr) + explicit ColumnDefine(ColId id_ = 0, String name_ = "", DataTypePtr type_ = nullptr, Field default_value_ = Field{}) : id(id_) , name(std::move(name_)) , type(std::move(type_)) , default_value(std::move(default_value_)) - , vector_index(vector_index_) {} }; diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index b05d77d2499..8cf629da88e 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -2022,7 +2022,7 @@ void DeltaMergeStore::applySchemaChanges(TiDB::TableInfo & table_info) std::atomic_store(&original_table_header, std::make_shared(toEmptyBlock(original_table_columns))); - // release the lock because `applyLocalIndexChange ` will try to acquire the lock + // release the lock because `applyLocalIndexChange` will try to acquire the lock // and generate tasks on segments lock.unlock(); diff --git a/dbms/src/Storages/DeltaMerge/Index/LocalIndexInfo.cpp b/dbms/src/Storages/DeltaMerge/Index/LocalIndexInfo.cpp index fe527ac7ad2..d73291e8202 100644 --- a/dbms/src/Storages/DeltaMerge/Index/LocalIndexInfo.cpp +++ b/dbms/src/Storages/DeltaMerge/Index/LocalIndexInfo.cpp @@ -26,40 +26,6 @@ extern const char force_not_support_vector_index[]; namespace DB::DM { -struct ComplexIndexID -{ - IndexID index_id; - ColumnID column_id; -}; - -bool operator==(const ComplexIndexID & lhs, const ComplexIndexID & rhs) -{ - return lhs.index_id == rhs.index_id && 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. @@ -141,48 +107,21 @@ LocalIndexInfosChangeset generateLocalIndexInfos( } // Keep a map of "indexes in existing_indexes" -> "offset in new_index_infos" - std::unordered_map original_local_index_id_map; + 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); + original_local_index_id_map.emplace(index.index_id, offset); new_index_infos->emplace_back(index); } } - std::unordered_set index_ids_in_new_table; - std::vector newly_added; - std::vector newly_dropped; - - // In the serverless branch, previously we define vector index on TiDB::ColumnInfo - for (const auto & col : new_table_info.columns) - { - 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); - } + std::unordered_set index_ids_in_new_table; + std::vector newly_added; + std::vector newly_dropped; for (const auto & idx : new_table_info.index_infos) { @@ -193,9 +132,7 @@ LocalIndexInfosChangeset generateLocalIndexInfos( 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 (!original_local_index_id_map.contains(idx.id)) { if (idx.state == TiDB::StatePublic || idx.state == TiDB::StateWriteReorganization) { @@ -206,15 +143,15 @@ LocalIndexInfosChangeset generateLocalIndexInfos( .column_id = column_id, .index_definition = idx.vector_index, }); - newly_added.emplace_back(cindex_id); - index_ids_in_new_table.emplace(cindex_id); + newly_added.emplace_back(idx.id); + index_ids_in_new_table.emplace(idx.id); } // else the index is not public or write reorg, consider this index as not exist } else { if (idx.state != TiDB::StateDeleteReorganization) - index_ids_in_new_table.emplace(cindex_id); + index_ids_in_new_table.emplace(idx.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 } @@ -244,12 +181,7 @@ LocalIndexInfosChangeset generateLocalIndexInfos( 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); - }, + [](const auto & id, FmtBuffer & fb) { fb.fmtAppend("index_id={}", id.first); }, ","); buf.append("]"); return buf.toString(); @@ -266,51 +198,28 @@ LocalIndexInfosChangeset generateLocalIndexInfos( 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); - }, + [](const auto & id, FmtBuffer & fb) { fb.fmtAppend("index_id={}", id.first); }, ","); 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); - }, + [](const auto & id, FmtBuffer & fb) { fb.fmtAppend("index_id={}", 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); - }, + [](const auto & id, FmtBuffer & fb) { fb.fmtAppend("index_id={}", id); }, ","); buf.append("]"); return buf.toString(); }; LOG_INFO(logger, "Local index info generated, {}", get_changed_logging()); - // only return the newly dropped index with index_id > EmptyIndexID - std::vector dropped_indexes; - for (const auto & i : newly_dropped) - { - if (i.index_id <= EmptyIndexID) - continue; - dropped_indexes.emplace_back(i.index_id); - } return LocalIndexInfosChangeset{ .new_local_index_infos = new_index_infos, - .dropped_indexes = std::move(dropped_indexes), + .dropped_indexes = std::move(newly_dropped), }; } 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 5dde081769c..6a9e9a4fb68 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_vector_index.cpp @@ -206,7 +206,7 @@ try { auto cols = DMTestEnv::getDefaultColumns(DMTestEnv::PkType::HiddenTiDBRowID, /*add_nullable*/ true); auto vec_cd = ColumnDefine(vec_column_id, vec_column_name, tests::typeFromString("Array(Float32)")); - vec_cd.vector_index = std::make_shared(TiDB::VectorIndexDefinition{ + auto vector_index = std::make_shared(TiDB::VectorIndexDefinition{ .kind = tipb::VectorIndexKind::HNSW, .dimension = 3, .distance_metric = tipb::VectorDistanceMetric::L2, @@ -229,7 +229,7 @@ try } dm_file = restoreDMFile(); - dm_file = buildIndex(*vec_cd.vector_index); + dm_file = buildIndex(*vector_index); // Read with exact match { @@ -836,7 +836,7 @@ try { auto cols = DMTestEnv::getDefaultColumns(DMTestEnv::PkType::HiddenTiDBRowID, /*add_nullable*/ true); auto vec_cd = ColumnDefine(vec_column_id, vec_column_name, tests::typeFromString("Array(Float32)")); - vec_cd.vector_index = std::make_shared(TiDB::VectorIndexDefinition{ + auto vector_index = std::make_shared(TiDB::VectorIndexDefinition{ .kind = tipb::VectorIndexKind::HNSW, .dimension = 3, .distance_metric = tipb::VectorDistanceMetric::L2, @@ -866,7 +866,7 @@ try } dm_file = restoreDMFile(); - dm_file = buildIndex(*vec_cd.vector_index); + dm_file = buildIndex(*vector_index); { auto ann_query_info = std::make_shared(); @@ -904,7 +904,7 @@ try { auto cols = DMTestEnv::getDefaultColumns(DMTestEnv::PkType::HiddenTiDBRowID, /*add_nullable*/ true); auto vec_cd = ColumnDefine(vec_column_id, vec_column_name, tests::typeFromString("Array(Float32)")); - vec_cd.vector_index = std::make_shared(TiDB::VectorIndexDefinition{ + auto vector_index = std::make_shared(TiDB::VectorIndexDefinition{ .kind = tipb::VectorIndexKind::HNSW, .dimension = 3, .distance_metric = tipb::VectorDistanceMetric::L2, @@ -933,7 +933,7 @@ try } dm_file = restoreDMFile(); - dm_file = buildIndex(*vec_cd.vector_index); + dm_file = buildIndex(*vector_index); // Pack #0 is filtered out according to VecIndex { @@ -1045,7 +1045,7 @@ try { auto cols = DMTestEnv::getDefaultColumns(DMTestEnv::PkType::HiddenTiDBRowID, /*add_nullable*/ true); auto vec_cd = ColumnDefine(vec_column_id, vec_column_name, tests::typeFromString("Array(Float32)")); - vec_cd.vector_index = std::make_shared(TiDB::VectorIndexDefinition{ + auto vector_index = std::make_shared(TiDB::VectorIndexDefinition{ .kind = tipb::VectorIndexKind::HNSW, .dimension = 1, .distance_metric = tipb::VectorDistanceMetric::L2, @@ -1076,7 +1076,7 @@ try } dm_file = restoreDMFile(); - dm_file = buildIndex(*vec_cd.vector_index); + dm_file = buildIndex(*vector_index); // Pack Filter using RowKeyRange { @@ -1223,11 +1223,6 @@ class VectorIndexSegmentTestBase void prepareColumns(const ColumnDefinesPtr & columns) override { auto vec_cd = ColumnDefine(vec_column_id, vec_column_name, tests::typeFromString("Array(Float32)")); - vec_cd.vector_index = std::make_shared(TiDB::VectorIndexDefinition{ - .kind = tipb::VectorIndexKind::HNSW, - .dimension = 1, - .distance_metric = tipb::VectorDistanceMetric::L2, - }); columns->emplace_back(vec_cd); } @@ -1752,9 +1747,7 @@ class VectorIndexSegmentOnS3Test FileCache::initialize(global_context.getPathCapacity(), file_cache_config); auto cols = DMTestEnv::getDefaultColumns(); - auto vec_cd = cdVec(); - vec_cd.vector_index = std::make_shared(index_info); - cols->emplace_back(vec_cd); + cols->emplace_back(cdVec()); setColumns(cols); auto dm_context = dmContext(); 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 4ea8ab1f4e5..536fd5c7586 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_local_index_info.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_local_index_info.cpp @@ -224,191 +224,4 @@ try } CATCH -TEST(LocalIndexInfoTest, CheckIndexAddWithVecIndexOnColumnInfo) -try -{ - // The serverless branch, vector index may directly defined on the ColumnInfo. - // Create table info with a vector index by column comments. - auto col_vector_index = TiDB::VectorIndexDefinitionPtr(new TiDB::VectorIndexDefinition{ - .kind = tipb::VectorIndexKind::HNSW, - .dimension = 3, - .distance_metric = tipb::VectorDistanceMetric::INNER_PRODUCT, - }); - TiDB::TableInfo table_info; - { - TiDB::ColumnInfo column_info; - column_info.name = "vec"; - column_info.id = 98; - table_info.columns.emplace_back(column_info); - - TiDB::ColumnInfo column_info_v1; - column_info_v1.name = "vec1"; - column_info_v1.id = 99; - column_info_v1.vector_index = col_vector_index; - table_info.columns.emplace_back(column_info_v1); - } - - // Add a vector index by add vector index dirctly. - 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); - } - - // check the different - auto logger = Logger::get(); - LocalIndexInfosPtr index_info = nullptr; - { - auto new_index_info = generateLocalIndexInfos(index_info, table_info, logger).new_local_index_infos; - ASSERT_NE(new_index_info, nullptr); - ASSERT_EQ(new_index_info->size(), 2); - - const auto & idx0 = (*new_index_info)[0]; - ASSERT_EQ(IndexType::Vector, idx0.type); - ASSERT_EQ(EmptyIndexID, idx0.index_id); // defined on TiDB::ColumnInfo - ASSERT_EQ(99, idx0.column_id); - ASSERT_NE(nullptr, idx0.index_definition); - ASSERT_EQ(col_vector_index->kind, idx0.index_definition->kind); - ASSERT_EQ(col_vector_index->dimension, idx0.index_definition->dimension); - ASSERT_EQ(col_vector_index->distance_metric, idx0.index_definition->distance_metric); - - const auto & idx1 = (*new_index_info)[1]; - ASSERT_EQ(IndexType::Vector, idx1.type); - ASSERT_EQ(expect_idx.id, idx1.index_id); - ASSERT_EQ(98, idx1.column_id); - ASSERT_NE(nullptr, idx1.index_definition); - ASSERT_EQ(expect_idx.vector_index->kind, idx1.index_definition->kind); - ASSERT_EQ(expect_idx.vector_index->dimension, idx1.index_definition->dimension); - ASSERT_EQ(expect_idx.vector_index->distance_metric, idx1.index_definition->distance_metric); - // check again, table_info.index_infos doesn't change and return them - LocalIndexInfosPtr empty_index_info = nullptr; - ASSERT_EQ(2, generateLocalIndexInfos(empty_index_info, table_info, logger).new_local_index_infos->size()); - // check again with the same table_info, nothing changed, return nullptr - ASSERT_EQ(nullptr, generateLocalIndexInfos(new_index_info, table_info, logger).new_local_index_infos); - - // update - index_info = new_index_info; - } - - // Drop the first vector index on column vec1. - table_info.index_infos.erase(table_info.index_infos.begin()); - - // Add another vector index to the TableInfo - TiDB::IndexInfo expect_idx2; - { - expect_idx2.id = 2; // another index_id - expect_idx2.idx_cols.emplace_back(default_index_col_info); - expect_idx2.vector_index = TiDB::VectorIndexDefinitionPtr(new TiDB::VectorIndexDefinition{ - .kind = tipb::VectorIndexKind::HNSW, - .dimension = 2, - .distance_metric = tipb::VectorDistanceMetric::COSINE, // another distance - }); - table_info.index_infos.emplace_back(expect_idx2); - } - // check the different - { - auto new_index_info = generateLocalIndexInfos(index_info, table_info, logger).new_local_index_infos; - ASSERT_NE(new_index_info, nullptr); - ASSERT_EQ(new_index_info->size(), 2); - - const auto & idx0 = (*new_index_info)[0]; - ASSERT_EQ(IndexType::Vector, idx0.type); - ASSERT_EQ(EmptyIndexID, idx0.index_id); // defined on TiDB::ColumnInfo - ASSERT_EQ(99, idx0.column_id); - ASSERT_NE(nullptr, idx0.index_definition); - ASSERT_EQ(col_vector_index->kind, idx0.index_definition->kind); - ASSERT_EQ(col_vector_index->dimension, idx0.index_definition->dimension); - ASSERT_EQ(col_vector_index->distance_metric, idx0.index_definition->distance_metric); - - const auto & idx1 = (*new_index_info)[1]; - ASSERT_EQ(IndexType::Vector, idx1.type); - ASSERT_EQ(expect_idx2.id, idx1.index_id); - ASSERT_EQ(98, idx1.column_id); - ASSERT_NE(nullptr, idx1.index_definition); - ASSERT_EQ(expect_idx2.vector_index->kind, idx1.index_definition->kind); - ASSERT_EQ(expect_idx2.vector_index->dimension, idx1.index_definition->dimension); - ASSERT_EQ(expect_idx2.vector_index->distance_metric, idx1.index_definition->distance_metric); - - // check again, nothing changed, return nullptr - ASSERT_EQ(nullptr, generateLocalIndexInfos(new_index_info, table_info, logger).new_local_index_infos); - } -} -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).new_local_index_infos; - 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).new_local_index_infos); - - // 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).new_local_index_infos; - 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).new_local_index_infos); - - // update - index_info = new_index_info; - } -} - } // namespace DB::DM::tests diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 43777c309a3..c9e17b5e16e 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -186,7 +186,6 @@ void StorageDeltaMerge::updateTableColumnInfo() if (itr != columns.end()) { col_def.default_value = itr->defaultValueToField(); - col_def.vector_index = itr->vector_index; } if (col_def.id != TiDBPkColumnID && col_def.id != VersionColumnID && col_def.id != DelMarkColumnID @@ -328,7 +327,7 @@ void StorageDeltaMerge::updateTableColumnInfo() table_column_defines.begin(), table_column_defines.end(), [](const ColumnDefine & col, FmtBuffer & fb) { - fb.fmtAppend("{} {} {}", col.name, col.type->getFamilyName(), col.vector_index); + fb.fmtAppend("{} {}", col.name, col.type->getFamilyName()); }, ", "); return fmt_buf.toString(); diff --git a/dbms/src/TiDB/Schema/TiDB.cpp b/dbms/src/TiDB/Schema/TiDB.cpp index ecd3b343840..57e2ee48c1f 100644 --- a/dbms/src/TiDB/Schema/TiDB.cpp +++ b/dbms/src/TiDB/Schema/TiDB.cpp @@ -117,31 +117,25 @@ enum class IndexType HNSW = 5, }; -VectorIndexDefinitionPtr parseVectorIndexFromJSON(IndexType index_type, const Poco::JSON::Object::Ptr & json) +inline tipb::VectorIndexKind toVectorIndexKind(IndexType index_type) { - assert(json); // not nullptr - - tipb::VectorIndexKind kind = tipb::VectorIndexKind::INVALID_INDEX_KIND; - if (unlikely(json->has("kind"))) - { - // TODO(vector-index): remove this deadcode - auto kind_field = json->getValue("kind"); - RUNTIME_CHECK_MSG( - tipb::VectorIndexKind_Parse(kind_field, &kind), - "invalid kind of vector index, {}", - kind_field); - RUNTIME_CHECK(kind != tipb::VectorIndexKind::INVALID_INDEX_KIND); - } - else + switch (index_type) { - RUNTIME_CHECK_MSG( - index_type == IndexType::HNSW, - "Invalid index_type for vector index, {}({})", - magic_enum::enum_name(index_type), - fmt::underlying(index_type)); - kind = tipb::VectorIndexKind::HNSW; + case IndexType::HNSW: + return tipb::VectorIndexKind::HNSW; + default: + throw Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Invalid index type for vector index {}", + magic_enum::enum_name(index_type)); } +} +VectorIndexDefinitionPtr parseVectorIndexFromJSON(IndexType index_type, const Poco::JSON::Object::Ptr & json) +{ + assert(json); // not nullptr + + auto kind = toVectorIndexKind(index_type); auto dimension = json->getValue("dimension"); RUNTIME_CHECK(dimension > 0 && dimension <= TiDB::MAX_VECTOR_DIMENSION, dimension); // Just a protection @@ -480,11 +474,6 @@ try } json->set("state", static_cast(state)); - if (vector_index) - { - json->set("vector_index", vectorIndexToJSON(vector_index)); - } - #ifndef NDEBUG // Check stringify in Debug mode std::stringstream str; @@ -536,12 +525,6 @@ try collate = type_json->get("Collate"); } state = static_cast(json->getValue("state")); - - // TODO(vector-index): remove this deadcode - if (auto vector_index_json = json->getObject("vector_index"); vector_index_json) - { - vector_index = parseVectorIndexFromJSON(IndexType::HNSW, vector_index_json); - } } catch (const Poco::Exception & e) { diff --git a/dbms/src/TiDB/Schema/TiDB.h b/dbms/src/TiDB/Schema/TiDB.h index cc40dfed1a6..67169149a60 100644 --- a/dbms/src/TiDB/Schema/TiDB.h +++ b/dbms/src/TiDB/Schema/TiDB.h @@ -127,9 +127,6 @@ struct ColumnInfo std::vector> elems; SchemaState state = StateNone; - // TODO(vector-index): This index will be moved to the table level later - VectorIndexDefinitionPtr vector_index = nullptr; - #ifdef M #error "Please undefine macro M first." #endif diff --git a/dbms/src/TiDB/Schema/tests/gtest_table_info.cpp b/dbms/src/TiDB/Schema/tests/gtest_table_info.cpp index 919a2e5596b..b7d5a824123 100644 --- a/dbms/src/TiDB/Schema/tests/gtest_table_info.cpp +++ b/dbms/src/TiDB/Schema/tests/gtest_table_info.cpp @@ -139,7 +139,7 @@ try { auto cases = { ParseCase{ - R"json({"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"col1","O":"col1"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":8}},{"default":null,"default_bit":null,"id":2,"name":{"L":"vec","O":"vec"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":225}}],"id":30,"index_info":[{"id":3,"idx_cols":[{"length":-1,"name":{"L":"vec","O":"vec"},"offset":0}],"idx_name":{"L":"idx1","O":"idx1"},"index_type":-1,"is_global":false,"is_invisible":false,"is_primary":false,"is_unique":false,"state":5,"vector_index":{"dimension":3,"distance_metric":"L2","kind":"HNSW"}}],"is_common_handle":false,"name":{"L":"t1","O":"t1"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":0,"update_timestamp":1723778704444603})json", + R"json({"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"col1","O":"col1"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":8}},{"default":null,"default_bit":null,"id":2,"name":{"L":"vec","O":"vec"},"offset":-1,"origin_default":null,"state":0,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":4097,"Flen":0,"Tp":225}}],"id":30,"index_info":[{"id":3,"idx_cols":[{"length":-1,"name":{"L":"vec","O":"vec"},"offset":0}],"idx_name":{"L":"idx1","O":"idx1"},"index_type":5,"is_global":false,"is_invisible":false,"is_primary":false,"is_unique":false,"state":5,"vector_index":{"dimension":3,"distance_metric":"L2"}}],"is_common_handle":false,"name":{"L":"t1","O":"t1"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":0,"update_timestamp":1723778704444603})json", [](const TableInfo & table_info) { ASSERT_EQ(table_info.index_infos.size(), 1); auto idx = table_info.index_infos[0]; @@ -163,25 +163,8 @@ try ASSERT_EQ(col1.id, 2); }, }, - // VectorIndex defined in the ColumnInfo ParseCase{ - R"json({"cols":[{"comment":"hnsw(distance=l2)","default":null,"default_bit":null,"id":1,"name":{"L":"v","O":"v"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":128,"Flen":5,"Tp":225},"vector_index":{"dimension":5,"distance_metric":"L2","kind":"HNSW"}}],"comment":"","id":96,"index_info":[],"is_common_handle":false,"keyspace_id":1,"name":{"L":"t","O":"t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Available":true,"Count":1},"update_timestamp":451956855279976452})json", - [](const TableInfo & table_info) { - ASSERT_EQ(table_info.index_infos.size(), 0); - ASSERT_EQ(table_info.columns.size(), 1); - auto col = table_info.columns[0]; - ASSERT_EQ(col.name, "v"); - ASSERT_EQ(col.tp, TiDB::TP::TypeTiDBVectorFloat32); - ASSERT_EQ(col.id, 1); - auto vector_index_on_col = col.vector_index; - ASSERT_NE(vector_index_on_col, nullptr); - ASSERT_EQ(vector_index_on_col->kind, tipb::VectorIndexKind::HNSW); - ASSERT_EQ(vector_index_on_col->dimension, 5); - ASSERT_EQ(vector_index_on_col->distance_metric, tipb::VectorDistanceMetric::L2); - }, - }, - ParseCase{ - R"json({"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"col","O":"col"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4099,"Flen":20,"Tp":8}},{"comment":"","default":null,"default_bit":null,"id":2,"name":{"L":"v","O":"v"},"offset":1,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":128,"Flen":5,"Tp":225}}],"comment":"","id":96,"index_info":[{"id":4,"idx_cols":[{"length":-1,"name":{"L":"v","O":"v"},"offset":1}],"idx_name":{"L":"idx_v_l2","O":"idx_v_l2"},"index_type":5,"is_global":false,"is_invisible":false,"is_primary":false,"is_unique":false,"state":3,"vector_index":{"dimension":5,"distance_metric":"L2","kind":"HNSW"}},{"id":3,"idx_cols":[{"length":-1,"name":{"L":"col","O":"col"},"offset":0}],"idx_name":{"L":"primary","O":"primary"},"index_type":1,"is_global":false,"is_invisible":false,"is_primary":true,"is_unique":true,"state":5}],"is_common_handle":false,"keyspace_id":1,"name":{"L":"ti","O":"ti"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Available":true,"Count":1},"update_timestamp":452024291984670725})json", + R"json({"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"col","O":"col"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4099,"Flen":20,"Tp":8}},{"comment":"","default":null,"default_bit":null,"id":2,"name":{"L":"v","O":"v"},"offset":1,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":128,"Flen":5,"Tp":225}}],"comment":"","id":96,"index_info":[{"id":4,"idx_cols":[{"length":-1,"name":{"L":"v","O":"v"},"offset":1}],"idx_name":{"L":"idx_v_l2","O":"idx_v_l2"},"index_type":5,"is_global":false,"is_invisible":false,"is_primary":false,"is_unique":false,"state":3,"vector_index":{"dimension":5,"distance_metric":"L2"}},{"id":3,"idx_cols":[{"length":-1,"name":{"L":"col","O":"col"},"offset":0}],"idx_name":{"L":"primary","O":"primary"},"index_type":1,"is_global":false,"is_invisible":false,"is_primary":true,"is_unique":true,"state":5}],"is_common_handle":false,"keyspace_id":1,"name":{"L":"ti","O":"ti"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Available":true,"Count":1},"update_timestamp":452024291984670725})json", [](const TableInfo & table_info) { // vector index && primary index // primary index alwasy be put at the first @@ -218,7 +201,6 @@ try ASSERT_EQ(col1.id, 2); }, }, - // VectorIndex defined without "kind" field ParseCase{ R"json({"Lock":null,"ShardRowIDBits":0,"auto_id_cache":0,"auto_inc_id":0,"auto_rand_id":0,"auto_random_bits":0,"auto_random_range_bits":0,"cache_table_status":0,"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"change_state_info":null,"comment":"","default":null,"default_bit":null,"default_is_expr":false,"dependences":null,"generated_expr_string":"","generated_stored":false,"hidden":false,"id":1,"name":{"L":"a","O":"a"},"offset":0,"origin_default":null,"origin_default_bit":null,"state":5,"type":{"Array":false,"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"ElemsIsBinaryLit":null,"Flag":4099,"Flen":11,"Tp":3},"version":2},{"change_state_info":null,"comment":"","default":null,"default_bit":null,"default_is_expr":false,"dependences":null,"generated_expr_string":"","generated_stored":false,"hidden":false,"id":2,"name":{"L":"vec","O":"vec"},"offset":1,"origin_default":null,"origin_default_bit":null,"state":5,"type":{"Array":false,"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"ElemsIsBinaryLit":null,"Flag":128,"Flen":3,"Tp":225},"version":2}],"comment":"","common_handle_version":0,"compression":"","constraint_info":null,"exchange_partition_info":null,"fk_info":null,"id":104, "index_info":[{"backfill_state":0,"comment":"","id":1,"idx_cols":[{"length":-1,"name":{"L":"vec","O":"vec"},"offset":1}],"idx_name":{"L":"v","O":"v"},"index_type":5,"is_global":false,"is_invisible":false,"is_primary":false,"is_unique":false,"mv_index":false,"state":3,"tbl_name":{"L":"","O":""},"vector_index":{"dimension":3,"distance_metric":"COSINE"}}],