From bad9338f20779a91d2e9c31dcb302bf433d5bd2b Mon Sep 17 00:00:00 2001 From: Vitaliy Filippov Date: Wed, 17 Sep 2025 16:06:20 +0000 Subject: [PATCH 1/4] Skip level table lookup if a prefix in a prefixed index has leaf cluster ID --- ydb/core/base/table_index.cpp | 2 - ydb/core/base/table_index.h | 2 + .../kqp/opt/logical/kqp_opt_log_indexes.cpp | 48 ++++++++++++++++--- 3 files changed, 44 insertions(+), 8 deletions(-) diff --git a/ydb/core/base/table_index.cpp b/ydb/core/base/table_index.cpp index d82aec792dd9..a88758196e8f 100644 --- a/ydb/core/base/table_index.cpp +++ b/ydb/core/base/table_index.cpp @@ -255,8 +255,6 @@ bool IsBuildImplTable(std::string_view tableName) { namespace NKMeans { -inline constexpr TClusterId PostingParentFlag = (1ull << 63ull); - bool HasPostingParentFlag(TClusterId parent) { return bool(parent & PostingParentFlag); } diff --git a/ydb/core/base/table_index.h b/ydb/core/base/table_index.h index 96e467fe2771..815166642b89 100644 --- a/ydb/core/base/table_index.h +++ b/ydb/core/base/table_index.h @@ -76,6 +76,8 @@ inline constexpr const char* IdColumnSequence = "__ydb_id_sequence"; inline constexpr const int DefaultKMeansRounds = 3; +inline constexpr TClusterId PostingParentFlag = (1ull << 63ull); + bool HasPostingParentFlag(TClusterId parent); void EnsureNoPostingParentFlag(TClusterId parent); TClusterId SetPostingParentFlag(TClusterId parent); diff --git a/ydb/core/kqp/opt/logical/kqp_opt_log_indexes.cpp b/ydb/core/kqp/opt/logical/kqp_opt_log_indexes.cpp index 062c6772db02..a2b8ac055c5c 100644 --- a/ydb/core/kqp/opt/logical/kqp_opt_log_indexes.cpp +++ b/ydb/core/kqp/opt/logical/kqp_opt_log_indexes.cpp @@ -642,7 +642,6 @@ TExprBase DoRewriteTopSortOverKMeansTree( YQL_ENSURE(levelTableDesc->Metadata->Name.EndsWith(NTableIndex::NKMeans::LevelTable)); YQL_ENSURE(postingTableDesc->Metadata->Name.EndsWith(NTableIndex::NKMeans::PostingTable)); - // TODO(mbkkt) It's kind of strange that almost everything here have same position const auto pos = match.Pos(); const auto levelTable = BuildTableMeta(*levelTableDesc->Metadata, pos, ctx); @@ -717,6 +716,33 @@ TExprBase DoRewriteTopSortOverKMeansTree( return TExprBase{read}; } +template +TExprBase FilterLeafRows(const TExprBase& read, TExprContext& ctx, TPositionHandle pos) { + auto leafFlag = Build(ctx, pos) + .Literal() + .Value(std::to_string(NTableIndex::NKMeans::PostingParentFlag)) // "9223372036854775808" + .Build() + .Done(); + auto prefixRowArg = ctx.NewArgument(pos, "prefixRow"); + auto prefixCluster = Build(ctx, pos) + .Struct(prefixRowArg) + .Name().Build(NTableIndex::NKMeans::ParentColumn) + .Done(); + return Build(ctx, pos) + .Input(read) + .Lambda() + .Args({prefixRowArg}) + .Body() + .Predicate() + .Left(prefixCluster) + .Right(leafFlag) + .Build() + .Value(prefixRowArg) + .Build() + .Build() + .Done(); +} + TExprBase DoRewriteTopSortOverPrefixedKMeansTree( const TReadMatch& match, const TCoFlatMap& flatMap, const TExprBase& lambdaArgs, const TExprBase& lambdaBody, const TCoTopBase& top, TExprContext& ctx, const TKqpOptimizeContext& kqpCtx, @@ -732,7 +758,6 @@ TExprBase DoRewriteTopSortOverPrefixedKMeansTree( YQL_ENSURE(postingTableDesc->Metadata->Name.EndsWith(NTableIndex::NKMeans::PostingTable)); YQL_ENSURE(prefixTableDesc->Metadata->Name.EndsWith(NTableIndex::NKMeans::PrefixTable)); - // TODO(mbkkt) It's kind of strange that almost everything here have same position const auto pos = match.Pos(); const auto levelTable = BuildTableMeta(*levelTableDesc->Metadata, pos, ctx); @@ -776,18 +801,29 @@ TExprBase DoRewriteTopSortOverPrefixedKMeansTree( .Lambda(prefixLambda) .Done().Ptr(); + read = Build(ctx, pos) + .Input(read) + .Done().Ptr(); + RemapIdToParent(ctx, pos, read); + auto prefixLeafRows = FilterLeafRows(TExprBase(read), ctx, pos); + auto prefixRootRows = FilterLeafRows(TExprBase(read), ctx, pos); + TKqpStreamLookupSettings settings; settings.Strategy = EStreamLookupStrategyType::LookupRows; - read = Build(ctx, pos) + auto levelRows = Build(ctx, pos) .Table(levelTable) - .LookupKeys(read) + .LookupKeys(prefixRootRows) .Columns(levelColumns) .Settings(settings.BuildNode(ctx, pos)) - .Done().Ptr(); + .Done().Ptr(); + VectorReadLevel(indexDesc, ctx, pos, kqpCtx, levelLambda, top, levelTable, levelColumns, levelRows); - VectorReadLevel(indexDesc, ctx, pos, kqpCtx, levelLambda, top, levelTable, levelColumns, read); + read = Build(ctx, pos) + .Add(levelRows) + .Add(prefixLeafRows) + .Done().Ptr(); VectorReadMain(ctx, pos, postingTable, postingTableDesc->Metadata, mainTable, tableDesc.Metadata, mainColumns, read); From 049c2cf484197e1d4603e0ff704140da65f775ae Mon Sep 17 00:00:00 2001 From: Vitaliy Filippov Date: Thu, 18 Sep 2025 17:00:32 +0000 Subject: [PATCH 2/4] Handle new prefixes in prefixed vector index update --- .../effects/kqp_opt_phy_effects_impl.h | 5 + .../effects/kqp_opt_phy_insert_index.cpp | 9 +- .../effects/kqp_opt_phy_upsert_index.cpp | 9 +- .../effects/kqp_opt_phy_vector_index.cpp | 373 ++++++++++++++++-- ydb/core/kqp/runtime/kqp_vector_actor.cpp | 8 +- .../kqp_indexes_prefixed_vector_ut.cpp | 121 +++++- 6 files changed, 471 insertions(+), 54 deletions(-) diff --git a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_effects_impl.h b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_effects_impl.h index de167865da32..95916f6cc9ad 100644 --- a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_effects_impl.h +++ b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_effects_impl.h @@ -113,4 +113,9 @@ NYql::NNodes::TExprBase BuildVectorIndexPrefixRows(const NYql::TKikimrTableDescr bool withData, const NYql::TIndexDescription* indexDesc, const NYql::NNodes::TExprBase& inputRows, TVector& indexTableColumns, NYql::TPositionHandle pos, NYql::TExprContext& ctx); +std::pair BuildVectorIndexPrefixRowsWithNew( + const NYql::TKikimrTableDescription& table, const NYql::TKikimrTableDescription& prefixTable, + const NYql::TIndexDescription* indexDesc, const NYql::NNodes::TExprBase& inputRows, + TVector& indexTableColumns, NYql::TPositionHandle pos, NYql::TExprContext& ctx); + } // NKikimr::NKqp::NOpt diff --git a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_insert_index.cpp b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_insert_index.cpp index 43069e0f010b..401757c79935 100644 --- a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_insert_index.cpp +++ b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_insert_index.cpp @@ -203,7 +203,14 @@ TExprBase KqpBuildInsertIndexStages(TExprBase node, TExprContext& ctx, const TKq // First resolve prefix IDs using StreamLookup const auto& prefixTable = kqpCtx.Tables->ExistingTable(kqpCtx.Cluster, TStringBuilder() << insert.Table().Path().Value() << "/" << indexDesc->Name << "/" << NKikimr::NTableIndex::NKMeans::PrefixTable); - upsertIndexRows = BuildVectorIndexPrefixRows(table, prefixTable, true, indexDesc, upsertIndexRows, indexTableColumns, insert.Pos(), ctx); + if (prefixTable.Metadata->Columns.at(NTableIndex::NKMeans::IdColumn).DefaultKind == NKikimrKqp::TKqpColumnMetadataProto::DEFAULT_KIND_SEQUENCE) { + auto res = BuildVectorIndexPrefixRowsWithNew(table, prefixTable, indexDesc, upsertIndexRows, indexTableColumns, insert.Pos(), ctx); + upsertIndexRows = std::move(res.first); + effects.emplace_back(std::move(res.second)); + } else { + // Handle old prefixed vector index tables without the sequence + upsertIndexRows = BuildVectorIndexPrefixRows(table, prefixTable, true, indexDesc, upsertIndexRows, indexTableColumns, insert.Pos(), ctx); + } } upsertIndexRows = BuildVectorIndexPostingRows(table, insert.Table(), indexDesc->Name, indexTableColumns, upsertIndexRows, true, insert.Pos(), ctx); diff --git a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_upsert_index.cpp b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_upsert_index.cpp index 3e8d722bc82b..2400160d42bb 100644 --- a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_upsert_index.cpp +++ b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_upsert_index.cpp @@ -920,7 +920,14 @@ TMaybeNode KqpPhyUpsertIndexEffectsImpl(TKqpPhyUpsertIndexMode mode, if (indexDesc->Type == TIndexDescription::EType::GlobalSyncVectorKMeansTree) { if (indexDesc->KeyColumns.size() > 1) { - upsertIndexRows = BuildVectorIndexPrefixRows(table, *prefixTable, true, indexDesc, upsertIndexRows, indexTableColumns, pos, ctx); + if (prefixTable->Metadata->Columns.at(NTableIndex::NKMeans::IdColumn).DefaultKind == NKikimrKqp::TKqpColumnMetadataProto::DEFAULT_KIND_SEQUENCE) { + auto res = BuildVectorIndexPrefixRowsWithNew(table, *prefixTable, indexDesc, upsertIndexRows, indexTableColumns, pos, ctx); + upsertIndexRows = std::move(res.first); + effects.emplace_back(std::move(res.second)); + } else { + // Handle old prefixed vector index tables without the sequence + upsertIndexRows = BuildVectorIndexPrefixRows(table, *prefixTable, true, indexDesc, upsertIndexRows, indexTableColumns, pos, ctx); + } } upsertIndexRows = BuildVectorIndexPostingRows(table, mainTableNode, diff --git a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_vector_index.cpp b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_vector_index.cpp index 748198e630f9..b426a0884be2 100644 --- a/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_vector_index.cpp +++ b/ydb/core/kqp/opt/physical/effects/kqp_opt_phy_vector_index.cpp @@ -143,15 +143,31 @@ TVector MakeColumnGetters(const TExprBase& rowArgument, const TVector return columnGetters; } -TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const TKikimrTableDescription& prefixTable, - bool withData, const TIndexDescription* indexDesc, const TExprBase& inputRows, - TVector& indexTableColumns, TPositionHandle pos, TExprContext& ctx) -{ - // This whole method does a very simple thing: - // SELECT i., p.__ydb_id AS __ydb_parent FROM i INNER JOIN prefixTable p ON p.=i. - // To implement it, we use a StreamLookup in Join mode. - // It takes tuples as input and returns tuples as output ("cookie" contains read stats). +TExprBase BuildNth(const TExprBase& lookupArg, const char *n, TPositionHandle pos, TExprContext& ctx) { + return Build(ctx, pos) + .Tuple(lookupArg) + .Index().Value(n).Build() + .Done(); +}; + +struct TVectorIndexPrefixLookup { + TExprBase Node; + TKqpTable PrefixTableNode; + TVector PrefixColumns; + const TStructExprType* PrefixType; + TVector PostingColumns; + TVectorIndexPrefixLookup(TExprBase&& node, TKqpTable&& prefixTableNode): + Node(std::move(node)), + PrefixTableNode(std::move(prefixTableNode)) { + } +}; + +TVectorIndexPrefixLookup BuildVectorIndexPrefixLookup( + const TKikimrTableDescription& table, const TKikimrTableDescription& prefixTable, + bool withData, bool withNulls, const TIndexDescription* indexDesc, const TExprBase& inputRows, + TPositionHandle pos, TExprContext& ctx) +{ TVector prefixColumns(indexDesc->KeyColumns.begin(), indexDesc->KeyColumns.end()-1); THashSet postingColumnsSet; TVector postingColumns; @@ -172,7 +188,15 @@ TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const } TKqpStreamLookupSettings streamLookupSettings; - streamLookupSettings.Strategy = EStreamLookupStrategyType::LookupJoinRows; + streamLookupSettings.Strategy = EStreamLookupStrategyType::LookupSemiJoinRows; + TVector leftColumns = postingColumns; + if (withNulls) { + for (const auto& column : prefixColumns) { + if (postingColumnsSet.emplace(column).second) { + leftColumns.emplace_back(column); + } + } + } TVector prefixItems; for (auto& column : prefixColumns) { @@ -183,17 +207,17 @@ TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const } auto prefixType = ctx.MakeType(prefixItems); - TVector postingItems; - for (auto& column : postingColumns) { + TVector leftItems; + for (auto& column : leftColumns) { auto type = table.GetColumnType(TString(column)); YQL_ENSURE(type, "No index column: " << column); auto itemType = ctx.MakeType(column, type); - postingItems.push_back(itemType); + leftItems.push_back(itemType); } - auto postingType = ctx.MakeType(postingItems); + auto leftType = ctx.MakeType(leftItems); TVector joinItemItems; - joinItemItems.push_back(postingType); + joinItemItems.push_back(leftType); joinItemItems.push_back(ctx.MakeType(prefixType)); auto joinItemType = ctx.MakeType(joinItemItems); auto joinInputType = ctx.MakeType(joinItemType); @@ -226,7 +250,7 @@ TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const // Join StreamLookup takes tuples as input - build them .Body() .Add() - .Add(MakeColumnGetters(rowArg, postingColumns, pos, ctx)) + .Add(MakeColumnGetters(rowArg, leftColumns, pos, ctx)) .Build() .Add() .Input() @@ -248,28 +272,284 @@ TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const .Settings(streamLookupSettings.BuildNode(ctx, pos)) .Done(); + TVectorIndexPrefixLookup res(std::move(lookup), std::move(prefixTableNode)); + res.PrefixColumns = std::move(prefixColumns); + res.PrefixType = prefixType; + res.PostingColumns = std::move(postingColumns); + return res; +} + +TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const TKikimrTableDescription& prefixTable, + bool withData, const TIndexDescription* indexDesc, const TExprBase& inputRows, + TVector& indexTableColumns, TPositionHandle pos, TExprContext& ctx) +{ + // This whole method does a very simple thing: + // SELECT i., p.__ydb_id AS __ydb_parent FROM i INNER JOIN prefixTable p ON p.=i. + // To implement it, we use a StreamLookup in Join mode. + // It takes tuples as input and returns tuples as output ("cookie" contains read stats). + + TVectorIndexPrefixLookup lookup = BuildVectorIndexPrefixLookup(table, prefixTable, withData, false, indexDesc, inputRows, pos, ctx); + // Join StreamLookup returns tuples as output // But we need left row + 1 field of the right row - build it using TCoMap const auto lookupArg = Build(ctx, pos).Name("lookupRow").Done(); - const auto leftRow = Build(ctx, pos) - .Tuple(lookupArg) - .Index().Value("0").Build() + const auto leftRow = BuildNth(lookupArg, "0", pos, ctx); + const auto rightRow = BuildNth(lookupArg, "1", pos, ctx); + + // Filter rows where rightRow is null + + auto mapLambda = Build(ctx, pos) + .Args({lookupArg}) + .Body() + .Predicate() + .Optional(rightRow) + .Build() + .Value() + .Input() + .Add(MakeColumnGetters(leftRow, lookup.PostingColumns, pos, ctx)) + .Add() + .Name().Build(NTableIndex::NKMeans::ParentColumn) + .template Value() + .Struct().Optional(rightRow).Build() + .Name().Build(NTableIndex::NKMeans::IdColumn) + .Build() + .Build() + .Build() + .Build() + .Build() .Done(); - const auto rightRow = Build(ctx, pos) - .Tuple(lookupArg) - .Index().Value("1").Build() + + auto mapStage = Build(ctx, pos) + .Inputs() + .Add(lookup.Node) + .Build() + .Program() + .Args({"rows"}) + .Body() + .Input() + .Input("rows") + .Lambda(mapLambda) + .Build() + .Build() + .Build() + .Settings().Build() + .Done(); + + indexTableColumns = std::move(lookup.PostingColumns); + indexTableColumns.push_back(NTableIndex::NKMeans::ParentColumn); + + return Build(ctx, pos) + .Output() + .Stage(mapStage) + .Index().Build(0) + .Build() + .Done(); +} + +std::pair BuildVectorIndexPrefixRowsWithNew( + const TKikimrTableDescription& table, const TKikimrTableDescription& prefixTable, + const TIndexDescription* indexDesc, const TExprBase& inputRows, + TVector& indexTableColumns, TPositionHandle pos, TExprContext& ctx) +{ + // This method is similar to the previous one, but also handles unknown prefixes. + // StreamLookupJoin is executed in SemiJoin mode in this case, so may + // be null for prefixes missing from the prefix table. + // We feed such rows to KqpCnSequencer, replicate their IDs to the output stream, + // and also separately insert them into the prefix table. + + TVectorIndexPrefixLookup lookup = BuildVectorIndexPrefixLookup(table, prefixTable, true, true, indexDesc, inputRows, pos, ctx); + + const auto lookupArg = Build(ctx, pos).Name("lookupRow").Done(); + const auto leftRow = BuildNth(lookupArg, "0", pos, ctx); + const auto rightRow = BuildNth(lookupArg, "1", pos, ctx); + + const auto newRowsArg = Build(ctx, pos).Name("rows").Done(); + auto newPrefixStream = Build(ctx, pos) + .Input(newRowsArg) + .Lambda() + .Args({lookupArg}) + .Body() + .Predicate() + .Value() + .Optional(rightRow) + .Build() + .Build() + .Value(leftRow) + .Build() + .Build() + .Done(); + + const auto keyArg = Build(ctx, pos).Name("keyArg").Done(); + const auto payloadArg = Build(ctx, pos).Name("payloadArg").Done(); + auto newPrefixDict = Build(ctx, pos) + .Stream(newPrefixStream) + .KeySelector() + .Args(keyArg) + .Body() + .Add(MakeColumnGetters(keyArg, lookup.PrefixColumns, pos, ctx)) + .Build() + .Build() + .PayloadSelector() + .Args(payloadArg) + .Body() + .Build() + .Build() + .Settings() + .Add().Build("One") + .Add().Build("Hashed") + .Build() + .Done(); + + auto newPrefixDictPrecompute = Build(ctx, pos) + .Connection() + .Output() + .Stage() + .Inputs() + .Add(lookup.Node) + .Build() + .Program() + .Args({newRowsArg}) + .Body(newPrefixDict) + .Build() + .Settings().Build() + .Build() + .Index().Build(0) + .Build() + .Build() + .Done(); + + // Feed newPrefixes from dict to KqpCnSequencer + + const auto dictArg = Build(ctx, pos).Name("dict").Done(); + auto newPrefixesStage = Build(ctx, pos) + .Inputs() + .Add(newPrefixDictPrecompute) + .Build() + .Program() + .Args({dictArg}) + .Body() + .Input() + .Dict(dictArg) + .Build() + .Build() + .Build() + .Settings().Build() + .Done(); + + auto listPrefixType = ctx.MakeType(lookup.PrefixType); + auto fullPrefixColumns = lookup.PrefixColumns; + fullPrefixColumns.push_back(NTableIndex::NKMeans::IdColumn); + auto fullPrefixColumnList = BuildColumnsList(fullPrefixColumns, pos, ctx); + + auto cnSequencer = Build(ctx, pos) + .Output() + .Stage(newPrefixesStage) + .Index().Build(0) + .Build() + .Table(lookup.PrefixTableNode) + .Columns(fullPrefixColumnList) + .DefaultConstraintColumns() + .Add(ctx.NewAtom(pos, NTableIndex::NKMeans::IdColumn)) + .Build() + .InputItemType(ExpandType(pos, *listPrefixType, ctx)) + .Done(); + + auto sequencerPrecompute = Build(ctx, pos) + .Connection() + .Output() + .Stage() + .Inputs() + .Add(cnSequencer) + .Build() + .Program() + .Args({"rows"}) + .Body() + .Input("rows") + .Build() + .Build() + .Settings().Build() + .Build() + .Index().Build(0) + .Build() + .Build() + .Done(); + + const auto sequencerArg = Build(ctx, pos).Name("seqRows").Done(); + const auto seqKeyArg = Build(ctx, pos).Name("seqKey").Done(); + const auto seqValueArg = Build(ctx, pos).Name("seqValue").Done(); + auto sequencerDict = Build(ctx, pos) + .List(sequencerArg) + .KeySelector() + .Args({seqKeyArg}) + .Body() + .Add(MakeColumnGetters(seqKeyArg, lookup.PrefixColumns, pos, ctx)) + .Build() + .Build() + .PayloadSelector() + .Args({seqValueArg}) + .Body() + .Struct(seqValueArg) + .Name().Build(NTableIndex::NKMeans::IdColumn) + .Build() + .Build() + .Settings() + .Add().Build("One") + .Add().Build("Hashed") + .Build() .Done(); + auto sequencerDictPrecompute = Build(ctx, pos) + .Connection() + .Output() + .Stage() + .Inputs() + .Add(sequencerPrecompute) + .Build() + .Program() + .Args({sequencerArg}) + .Body() + .Input() + .Add(sequencerDict) + .Build() + .Build() + .Build() + .Settings().Build() + .Build() + .Index().Build(0) + .Build() + .Build() + .Done(); + + // Take output and remap it to the input using a dictionary + + const auto sequencerDictArg = Build(ctx, pos).Name("dict").Done(); + const auto origArg = Build(ctx, pos).Name("origRows").Done(); + const auto fillRowArg = Build(ctx, pos).Name("fillRow").Done(); + const auto leftFillRow = BuildNth(fillRowArg, "0", pos, ctx); + const auto rightFillRow = BuildNth(fillRowArg, "1", pos, ctx); auto mapLambda = Build(ctx, pos) - .Args({lookupArg}) + .Args({fillRowArg}) .Body() - .Add(MakeColumnGetters(leftRow, postingColumns, pos, ctx)) + .Add(MakeColumnGetters(leftFillRow, lookup.PostingColumns, pos, ctx)) .Add() .Name().Build(NTableIndex::NKMeans::ParentColumn) - .template Value() - .Struct().Optional(rightRow).Build() - .Name().Build(NTableIndex::NKMeans::IdColumn) + .template Value() + .Predicate() + .Optional(rightFillRow) + .Build() + .ThenValue() + .Struct().Optional(rightFillRow).Build() + .Name().Build(NTableIndex::NKMeans::IdColumn) + .Build() + .ElseValue() + .Optional() + .Collection(sequencerDictArg) + .Lookup() + .Add(MakeColumnGetters(leftFillRow, lookup.PrefixColumns, pos, ctx)) + .Build() + .Build() + .Build() .Build() .Build() .Build() @@ -277,13 +557,14 @@ TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const auto mapStage = Build(ctx, pos) .Inputs() - .Add(lookup) + .Add(lookup.Node) + .Add(sequencerDictPrecompute) .Build() .Program() - .Args({"rows"}) + .Args({origArg, sequencerDictArg}) .Body() .Input() - .Input("rows") + .Input(origArg) .Lambda(mapLambda) .Build() .Build() @@ -291,15 +572,43 @@ TExprBase BuildVectorIndexPrefixRows(const TKikimrTableDescription& table, const .Settings().Build() .Done(); - postingColumns.push_back(NTableIndex::NKMeans::ParentColumn); - indexTableColumns = std::move(postingColumns); + indexTableColumns = std::move(lookup.PostingColumns); + indexTableColumns.push_back(NTableIndex::NKMeans::ParentColumn); - return Build(ctx, pos) + auto mappedRows = Build(ctx, pos) .Output() .Stage(mapStage) .Index().Build(0) .Build() .Done(); + + auto upsertPrefixStage = Build(ctx, pos) + .Inputs() + .Add(sequencerPrecompute) + .Build() + .Program() + .Args({"rows"}) + .Body() + .Input("rows") + .Build() + .Build() + .Settings().Build() + .Done(); + + auto upsertPrefix = Build(ctx, pos) + .Table(lookup.PrefixTableNode) + .Input() + .Output() + .Stage(upsertPrefixStage) + .Index().Build(0) + .Build() + .Build() + .Columns(fullPrefixColumnList) + .ReturningColumns().Build() + .IsBatch(ctx.NewAtom(pos, "false")) + .Done(); + + return std::make_pair((TExprBase)mappedRows, (TExprBase)upsertPrefix); } } // namespace NKikimr::NKqp::NOpt diff --git a/ydb/core/kqp/runtime/kqp_vector_actor.cpp b/ydb/core/kqp/runtime/kqp_vector_actor.cpp index 4069e1228ad5..6133778ecaa5 100644 --- a/ydb/core/kqp/runtime/kqp_vector_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_vector_actor.cpp @@ -185,9 +185,12 @@ class TKqpVectorResolveActor : public NActors::TActorBootstrapped 0) { diff --git a/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp b/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp index 8529a8b7800b..26a484fae2c2 100644 --- a/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp +++ b/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp @@ -62,7 +62,7 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { } } - void DoPositiveQueriesVectorIndex(TSession& session, const TString& mainQuery, const TString& indexQuery, bool covered = false) { + void DoPositiveQueriesVectorIndex(TSession& session, const TString& mainQuery, const TString& indexQuery, bool covered = false, size_t count = 3) { auto toStr = [](const auto& rs) -> TString { TStringBuilder b; for (const auto& r : rs) { @@ -72,12 +72,12 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { }; auto mainResults = DoPositiveQueryVectorIndex(session, mainQuery); absl::c_sort(mainResults); - UNIT_ASSERT_EQUAL_C(mainResults.size(), 3, toStr(mainResults)); + UNIT_ASSERT_EQUAL_C(mainResults.size(), count, toStr(mainResults)); UNIT_ASSERT_C(std::unique(mainResults.begin(), mainResults.end()) == mainResults.end(), toStr(mainResults)); auto indexResults = DoPositiveQueryVectorIndex(session, indexQuery, covered); absl::c_sort(indexResults); - UNIT_ASSERT_EQUAL_C(indexResults.size(), 3, toStr(indexResults)); + UNIT_ASSERT_EQUAL_C(indexResults.size(), count, toStr(indexResults)); UNIT_ASSERT_C(std::unique(indexResults.begin(), indexResults.end()) == indexResults.end(), toStr(indexResults)); UNIT_ASSERT_VALUES_EQUAL(mainResults, indexResults); @@ -89,10 +89,9 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { std::string_view direction, std::string_view left, std::string_view right, - bool covered = false) { - constexpr std::string_view init = - "$target = \"\x67\x68\x03\";\n" - "$user = \"user_b\";"; + bool covered = false, + std::string_view init = "$target = \"\x67\x68\x03\";\n$user = \"user_b\";", + size_t count = 3) { std::string metric = std::format("Knn::{}({}, {})", function, left, right); // no metric in result { @@ -113,7 +112,7 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { ORDER BY {} {} LIMIT 3; )", init, metric, direction))); - DoPositiveQueriesVectorIndex(session, plainQuery, indexQuery, covered); + DoPositiveQueriesVectorIndex(session, plainQuery, indexQuery, covered, count); } // metric in result { @@ -130,7 +129,7 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { ORDER BY {} {} LIMIT 3; )", init, metric, metric, direction))); - DoPositiveQueriesVectorIndex(session, plainQuery, indexQuery, covered); + DoPositiveQueriesVectorIndex(session, plainQuery, indexQuery, covered, count); } // metric as result // TODO(mbkkt) fix this behavior too @@ -149,7 +148,7 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { ORDER BY m {} LIMIT 3; )", init, metric, direction))); - DoPositiveQueriesVectorIndex(session, plainQuery, indexQuery, covered); + DoPositiveQueriesVectorIndex(session, plainQuery, indexQuery, covered, count); } } @@ -157,20 +156,23 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { TSession& session, std::string_view function, std::string_view direction, - bool covered = false) { + bool covered = false, + std::string_view init = "$target = \"\x67\x68\x03\";\n$user = \"user_b\";", + size_t count = 3) { // target is left, member is right - DoPositiveQueriesPrefixedVectorIndexOrderBy(session, function, direction, "$target", "emb", covered); + DoPositiveQueriesPrefixedVectorIndexOrderBy(session, function, direction, "$target", "emb", covered, init, count); // target is right, member is left - DoPositiveQueriesPrefixedVectorIndexOrderBy(session, function, direction, "emb", "$target", covered); + DoPositiveQueriesPrefixedVectorIndexOrderBy(session, function, direction, "emb", "$target", covered, init, count); } - void DoPositiveQueriesPrefixedVectorIndexOrderByCosine(TSession& session, bool covered = false) { + void DoPositiveQueriesPrefixedVectorIndexOrderByCosine(TSession& session, bool covered = false, + std::string_view init = "$target = \"\x67\x68\x03\";\n$user = \"user_b\";", size_t count = 3) { // distance, default direction - DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineDistance", "", covered); + DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineDistance", "", covered, init, count); // distance, asc direction - DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineDistance", "ASC", covered); + DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineDistance", "ASC", covered, init, count); // similarity, desc direction - DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineSimilarity", "DESC", covered); + DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineSimilarity", "DESC", covered, init, count); } TSession DoCreateTableForPrefixedVectorIndex(TTableClient& db, bool nullable, bool suffixPk = false) { @@ -567,6 +569,91 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { DoTestPrefixedVectorIndexInsert(Returning, Covered); } + Y_UNIT_TEST_QUAD(PrefixedVectorIndexInsertNewPrefix, Nullable, Covered) { + NKikimrConfig::TFeatureFlags featureFlags; + featureFlags.SetEnableVectorIndex(true); + auto setting = NKikimrKqp::TKqpSetting(); + auto serverSettings = TKikimrSettings() + .SetFeatureFlags(featureFlags) + .SetKqpSettings({setting}); + + TKikimrRunner kikimr(serverSettings); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::BUILD_INDEX, NActors::NLog::PRI_TRACE); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::SEQUENCESHARD, NActors::NLog::PRI_TRACE); + + auto db = kikimr.GetTableClient(); + + auto session = DoCreateTableForPrefixedVectorIndex(db, Nullable); + DoCreatePrefixedVectorIndex(session, false, Covered ? "COVER (user, emb, data)" : "", 2); + + const TString originalPostingTable = ReadTablePartToYson(session, "/Root/TestTable/index/indexImplPostingTable"); + + // Insert to the table with index should succeed + { + TString query1(Q_(R"( + INSERT INTO `/Root/TestTable` (pk, user, emb, data) VALUES + (101, "user_a", "\x03\x29\x03", "101"), + (102, "user_xxx", "\x03\x29\x03", "102"), + (111, "user_yyy", "\x76\x75\x03", "111"), + (112, "user_yyy", "\x76\x75\x03", "112"); + )")); + + auto result = session.ExecuteDataQuery(query1, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()) + .ExtractValueSync(); + UNIT_ASSERT(result.IsSuccess()); + } + + // Index is updated + const TString postingTable1_ins = ReadTablePartToYson(session, "/Root/TestTable/index/indexImplPostingTable"); + UNIT_ASSERT_STRINGS_UNEQUAL(originalPostingTable, postingTable1_ins); + + // Check that we can now actually find new rows + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_a\";"); + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_b\";"); + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_xxx\";", 1); + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_yyy\";", 2); + + // Check that PKs 1/101, 111/112 are now in same clusters + { + const TString query1(Q_(R"( + SELECT COUNT(DISTINCT __ydb_parent) FROM `/Root/TestTable/index/indexImplPostingTable` + WHERE pk IN (1, 101) + UNION ALL + SELECT COUNT(DISTINCT __ydb_parent) FROM `/Root/TestTable/index/indexImplPostingTable` + WHERE pk IN (111, 112) + ; + )")); + auto result = session.ExecuteDataQuery(query1, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()) + .ExtractValueSync(); + UNIT_ASSERT(result.IsSuccess()); + UNIT_ASSERT_VALUES_EQUAL(NYdb::FormatResultSetYson(result.GetResultSet(0)), "[[1u];[1u]]"); + } + + // Delete one of the new rows + { + auto result = session.ExecuteDataQuery("DELETE FROM `/Root/TestTable` WHERE pk=112;", + TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).ExtractValueSync(); + UNIT_ASSERT(result.IsSuccess()); + } + + // Check that PK 112 is not present in the posting table + { + const TString query1(Q_(R"( + SELECT COUNT(*) FROM `/Root/TestTable/index/indexImplPostingTable` + WHERE pk=112; + )")); + auto result = session.ExecuteDataQuery(query1, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()) + .ExtractValueSync(); + UNIT_ASSERT(result.IsSuccess()); + UNIT_ASSERT_VALUES_EQUAL(NYdb::FormatResultSetYson(result.GetResultSet(0)), "[[0u]]"); + } + } + void DoTestPrefixedVectorIndexDelete(const TString& deleteQuery, bool returning, bool covered) { NKikimrConfig::TFeatureFlags featureFlags; featureFlags.SetEnableVectorIndex(true); From b1fbf6b375c881da455e3136a2dbd978a0e2f97e Mon Sep 17 00:00:00 2001 From: Vitaliy Filippov Date: Thu, 25 Sep 2025 11:09:21 +0000 Subject: [PATCH 3/4] Do not try to update vector indexes until their build is finished --- ydb/core/kqp/provider/yql_kikimr_gateway.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ydb/core/kqp/provider/yql_kikimr_gateway.h b/ydb/core/kqp/provider/yql_kikimr_gateway.h index b3a68c15083a..67f252f3e7a6 100644 --- a/ydb/core/kqp/provider/yql_kikimr_gateway.h +++ b/ydb/core/kqp/provider/yql_kikimr_gateway.h @@ -257,6 +257,10 @@ struct TIndexDescription { case EType::GlobalAsync: return false; case EType::GlobalSyncVectorKMeansTree: + if (State != EIndexState::Ready) { + // Do not try to update vector indexes until their build is finished + return false; + } return true; case EType::GlobalFulltext: return true; From 3d7b628a458fd86c462f510c8ce18fee42bd4e95 Mon Sep 17 00:00:00 2001 From: Vitaliy Filippov Date: Thu, 25 Sep 2025 13:25:01 +0000 Subject: [PATCH 4/4] Add tests for inserting into an empty prefixed vector index --- .../kqp_indexes_prefixed_vector_ut.cpp | 179 +++++++++++++----- 1 file changed, 135 insertions(+), 44 deletions(-) diff --git a/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp b/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp index 26a484fae2c2..c28c75e2df23 100644 --- a/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp +++ b/ydb/core/kqp/ut/indexes/kqp_indexes_prefixed_vector_ut.cpp @@ -175,7 +175,8 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { DoPositiveQueriesPrefixedVectorIndexOrderBy(session, "CosineSimilarity", "DESC", covered, init, count); } - TSession DoCreateTableForPrefixedVectorIndex(TTableClient& db, bool nullable, bool suffixPk = false) { + TSession DoOnlyCreateTableForPrefixedVectorIndex(TTableClient& db, bool nullable, bool suffixPk = false, + bool withIndex = false, bool covered = false) { auto session = db.CreateSession().GetValueSync().GetSession(); { @@ -212,54 +213,74 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { .AppendSplitPoints(TValueBuilder{}.BeginTuple().AddElement().OptionalInt64(60).EndTuple().Build()); tableBuilder.SetPartitionAtKeys(partitions); } + if (withIndex) { + TKMeansTreeSettings kmeans; + kmeans.Settings.Metric = TVectorIndexSettings::EMetric::CosineDistance; + kmeans.Settings.VectorType = TVectorIndexSettings::EVectorType::Uint8; + kmeans.Settings.VectorDimension = 2; + kmeans.Clusters = 2; + kmeans.Levels = 2; + std::vector dataColumns; + if (covered) { + dataColumns = {"user", "emb", "data"}; + } + tableBuilder.AddVectorKMeansTreeIndex("index", {"user", "emb"}, dataColumns, kmeans); + } auto result = session.CreateTable("/Root/TestTable", tableBuilder.Build()).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL(result.IsTransportError(), false); UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); } - { - const TString query1(Q_(R"( - UPSERT INTO `/Root/TestTable` (pk, user, emb, data) VALUES)" - "( 1, \"user_a\", \"\x03\x30\x03\", \"10\")," - "(11, \"user_a\", \"\x13\x31\x03\", \"11\")," - "(21, \"user_a\", \"\x23\x32\x03\", \"12\")," - "(31, \"user_a\", \"\x53\x33\x03\", \"13\")," - "(41, \"user_a\", \"\x43\x34\x03\", \"14\")," - "(51, \"user_a\", \"\x50\x60\x03\", \"15\")," - "(61, \"user_a\", \"\x61\x61\x03\", \"16\")," - "(71, \"user_a\", \"\x12\x62\x03\", \"17\")," - "(81, \"user_a\", \"\x75\x76\x03\", \"18\")," - "(91, \"user_a\", \"\x76\x76\x03\", \"19\")," - - "( 2, \"user_b\", \"\x03\x30\x03\", \"20\")," - "(12, \"user_b\", \"\x13\x31\x03\", \"21\")," - "(22, \"user_b\", \"\x23\x32\x03\", \"22\")," - "(32, \"user_b\", \"\x53\x33\x03\", \"23\")," - "(42, \"user_b\", \"\x43\x34\x03\", \"24\")," - "(52, \"user_b\", \"\x50\x60\x03\", \"25\")," - "(62, \"user_b\", \"\x61\x61\x03\", \"26\")," - "(72, \"user_b\", \"\x12\x62\x03\", \"27\")," - "(82, \"user_b\", \"\x75\x76\x03\", \"28\")," - "(92, \"user_b\", \"\x76\x76\x03\", \"29\")," - - "( 3, \"user_c\", \"\x03\x30\x03\", \"30\")," - "(13, \"user_c\", \"\x13\x31\x03\", \"31\")," - "(23, \"user_c\", \"\x23\x32\x03\", \"32\")," - "(33, \"user_c\", \"\x53\x33\x03\", \"33\")," - "(43, \"user_c\", \"\x43\x34\x03\", \"34\")," - "(53, \"user_c\", \"\x50\x60\x03\", \"35\")," - "(63, \"user_c\", \"\x61\x61\x03\", \"36\")," - "(73, \"user_c\", \"\x12\x62\x03\", \"37\")," - "(83, \"user_c\", \"\x75\x76\x03\", \"38\")," - "(93, \"user_c\", \"\x76\x76\x03\", \"39\");" - )); - - auto result = session.ExecuteDataQuery( - query1, - TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()) - .ExtractValueSync(); - UNIT_ASSERT_C(result.IsSuccess(), result.GetIssues().ToString()); - } + return session; + } + + void InsertDataForPrefixedVectorIndex(TSession& session) { + const TString query1(Q_(R"( + UPSERT INTO `/Root/TestTable` (pk, user, emb, data) VALUES)" + "( 1, \"user_a\", \"\x03\x30\x03\", \"10\")," + "(11, \"user_a\", \"\x13\x31\x03\", \"11\")," + "(21, \"user_a\", \"\x23\x32\x03\", \"12\")," + "(31, \"user_a\", \"\x53\x33\x03\", \"13\")," + "(41, \"user_a\", \"\x43\x34\x03\", \"14\")," + "(51, \"user_a\", \"\x50\x60\x03\", \"15\")," + "(61, \"user_a\", \"\x61\x61\x03\", \"16\")," + "(71, \"user_a\", \"\x12\x62\x03\", \"17\")," + "(81, \"user_a\", \"\x75\x76\x03\", \"18\")," + "(91, \"user_a\", \"\x76\x76\x03\", \"19\")," + + "( 2, \"user_b\", \"\x03\x30\x03\", \"20\")," + "(12, \"user_b\", \"\x13\x31\x03\", \"21\")," + "(22, \"user_b\", \"\x23\x32\x03\", \"22\")," + "(32, \"user_b\", \"\x53\x33\x03\", \"23\")," + "(42, \"user_b\", \"\x43\x34\x03\", \"24\")," + "(52, \"user_b\", \"\x50\x60\x03\", \"25\")," + "(62, \"user_b\", \"\x61\x61\x03\", \"26\")," + "(72, \"user_b\", \"\x12\x62\x03\", \"27\")," + "(82, \"user_b\", \"\x75\x76\x03\", \"28\")," + "(92, \"user_b\", \"\x76\x76\x03\", \"29\")," + + "( 3, \"user_c\", \"\x03\x30\x03\", \"30\")," + "(13, \"user_c\", \"\x13\x31\x03\", \"31\")," + "(23, \"user_c\", \"\x23\x32\x03\", \"32\")," + "(33, \"user_c\", \"\x53\x33\x03\", \"33\")," + "(43, \"user_c\", \"\x43\x34\x03\", \"34\")," + "(53, \"user_c\", \"\x50\x60\x03\", \"35\")," + "(63, \"user_c\", \"\x61\x61\x03\", \"36\")," + "(73, \"user_c\", \"\x12\x62\x03\", \"37\")," + "(83, \"user_c\", \"\x75\x76\x03\", \"38\")," + "(93, \"user_c\", \"\x76\x76\x03\", \"39\");" + )); + + auto result = session.ExecuteDataQuery( + query1, + TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()) + .ExtractValueSync(); + UNIT_ASSERT_C(result.IsSuccess(), result.GetIssues().ToString()); + } + + TSession DoCreateTableForPrefixedVectorIndex(TTableClient& db, bool nullable, bool suffixPk = false) { + auto session = DoOnlyCreateTableForPrefixedVectorIndex(db, nullable, suffixPk); + InsertDataForPrefixedVectorIndex(session); return session; } @@ -654,6 +675,76 @@ Y_UNIT_TEST_SUITE(KqpPrefixedVectorIndexes) { } } + Y_UNIT_TEST_QUAD(PrefixedVectorEmptyIndexedTableInsert, Nullable, Covered) { + NKikimrConfig::TFeatureFlags featureFlags; + featureFlags.SetEnableVectorIndex(true); + auto setting = NKikimrKqp::TKqpSetting(); + auto serverSettings = TKikimrSettings() + .SetFeatureFlags(featureFlags) + .SetKqpSettings({setting}); + + TKikimrRunner kikimr(serverSettings); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::BUILD_INDEX, NActors::NLog::PRI_TRACE); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::SEQUENCESHARD, NActors::NLog::PRI_TRACE); + + auto db = kikimr.GetTableClient(); + + auto session = DoOnlyCreateTableForPrefixedVectorIndex(db, Nullable, false, true, Covered); + + const TString originalPostingTable = ReadTablePartToYson(session, "/Root/TestTable/index/indexImplPostingTable"); + UNIT_ASSERT_STRINGS_EQUAL(originalPostingTable, "[]"); + + // Insert to the table with index should succeed + InsertDataForPrefixedVectorIndex(session); + + // Index is updated + const TString postingTable1_ins = ReadTablePartToYson(session, "/Root/TestTable/index/indexImplPostingTable"); + UNIT_ASSERT_STRINGS_UNEQUAL(originalPostingTable, postingTable1_ins); + + // Check that we can now actually find new rows + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_a\";"); + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_b\";"); + } + + // Same as PrefixedVectorEmptyIndexedTableInsert, but the index is created separately after creating the table + Y_UNIT_TEST_QUAD(EmptyPrefixedVectorIndexInsert, Nullable, Covered) { + NKikimrConfig::TFeatureFlags featureFlags; + featureFlags.SetEnableVectorIndex(true); + auto setting = NKikimrKqp::TKqpSetting(); + auto serverSettings = TKikimrSettings() + .SetFeatureFlags(featureFlags) + .SetKqpSettings({setting}); + + TKikimrRunner kikimr(serverSettings); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::BUILD_INDEX, NActors::NLog::PRI_TRACE); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::SEQUENCESHARD, NActors::NLog::PRI_TRACE); + + auto db = kikimr.GetTableClient(); + + auto session = DoOnlyCreateTableForPrefixedVectorIndex(db, Nullable, false); + DoCreatePrefixedVectorIndex(session, false, Covered ? "COVER (user, emb, data)" : "", 2); + + const TString originalPostingTable = ReadTablePartToYson(session, "/Root/TestTable/index/indexImplPostingTable"); + UNIT_ASSERT_STRINGS_EQUAL(originalPostingTable, "[]"); + + // Insert to the table with index should succeed + InsertDataForPrefixedVectorIndex(session); + + // Index is updated + const TString postingTable1_ins = ReadTablePartToYson(session, "/Root/TestTable/index/indexImplPostingTable"); + UNIT_ASSERT_STRINGS_UNEQUAL(originalPostingTable, postingTable1_ins); + + // Check that we can now actually find new rows + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_a\";"); + DoPositiveQueriesPrefixedVectorIndexOrderByCosine(session, Covered, + "$target = \"\x67\x68\x03\";\n$user = \"user_b\";"); + } + void DoTestPrefixedVectorIndexDelete(const TString& deleteQuery, bool returning, bool covered) { NKikimrConfig::TFeatureFlags featureFlags; featureFlags.SetEnableVectorIndex(true);