Skip to content

Commit 932db4f

Browse files
fix(kqp): disable stream join for joins with non-pk columns in predicate (#12573)
1 parent 7f6d8cc commit 932db4f

File tree

10 files changed

+247
-14
lines changed

10 files changed

+247
-14
lines changed

ydb/core/kqp/common/kqp_yql.cpp

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -509,10 +509,11 @@ NNodes::TCoNameValueTupleList TKqpStreamLookupSettings::BuildNode(TExprContext&
509509
.Done()
510510
);
511511

512-
if (AllowNullKeys) {
512+
if (AllowNullKeysPrefixSize) {
513513
settings.emplace_back(
514514
Build<TCoNameValueTuple>(ctx, pos)
515515
.Name().Build(AllowNullKeysSettingName)
516+
.Value<TCoAtom>().Build(ToString(*AllowNullKeysPrefixSize))
516517
.Done()
517518
);
518519
}
@@ -543,7 +544,8 @@ TKqpStreamLookupSettings TKqpStreamLookupSettings::Parse(const NNodes::TCoNameVa
543544
YQL_ENSURE(tuple.Value().Maybe<TCoAtom>());
544545
settings.Strategy = getLookupStrategyType(tuple.Value().Cast<TCoAtom>().Value());
545546
} else if (name == AllowNullKeysSettingName) {
546-
settings.AllowNullKeys = true;
547+
YQL_ENSURE(tuple.Value().Maybe<TCoAtom>());
548+
settings.AllowNullKeysPrefixSize = FromString<ui32>(tuple.Value().Cast<TCoAtom>().Value());
547549
} else {
548550
YQL_ENSURE(false, "Unknown KqpStreamLookup setting name '" << name << "'");
549551
}

ydb/core/kqp/common/kqp_yql.h

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -55,14 +55,14 @@ enum class EStreamLookupStrategyType {
5555

5656
struct TKqpStreamLookupSettings {
5757
static constexpr TStringBuf StrategySettingName = "Strategy";
58-
static constexpr TStringBuf AllowNullKeysSettingName = "AllowNullKeys";
58+
static constexpr TStringBuf AllowNullKeysSettingName = "AllowNullKeysPrefixSize";
5959

6060
// stream lookup strategy types
6161
static constexpr std::string_view LookupStrategyName = "LookupRows"sv;
6262
static constexpr std::string_view LookupJoinStrategyName = "LookupJoinRows"sv;
6363
static constexpr std::string_view LookupSemiJoinStrategyName = "LookupSemiJoinRows"sv;
6464

65-
bool AllowNullKeys = false;
65+
TMaybe<ui32> AllowNullKeysPrefixSize;
6666
EStreamLookupStrategyType Strategy = EStreamLookupStrategyType::Unspecified;
6767

6868
NNodes::TCoNameValueTupleList BuildNode(TExprContext& ctx, TPositionHandle pos) const;

ydb/core/kqp/executer_actor/kqp_tasks_graph.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -420,7 +420,7 @@ void BuildStreamLookupChannels(TKqpTasksGraph& graph, const TStageInfo& stageInf
420420

421421
settings->SetLookupStrategy(streamLookup.GetLookupStrategy());
422422
settings->SetKeepRowsOrder(streamLookup.GetKeepRowsOrder());
423-
settings->SetAllowNullKeys(streamLookup.GetAllowNullKeys());
423+
settings->SetAllowNullKeysPrefixSize(streamLookup.GetAllowNullKeysPrefixSize());
424424

425425
TTransform streamLookupTransform;
426426
streamLookupTransform.Type = "StreamLookupInputTransformer";

ydb/core/kqp/opt/logical/kqp_opt_log_indexes.cpp

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -424,7 +424,6 @@ TExprBase KqpRewriteStreamLookupIndex(const TExprBase& node, TExprContext& ctx,
424424

425425
auto streamLookupIndex = node.Maybe<TKqlStreamLookupIndex>().Cast();
426426
auto settings = TKqpStreamLookupSettings::Parse(streamLookupIndex);
427-
settings.AllowNullKeys = true;
428427

429428
const auto& tableDesc = GetTableData(*kqpCtx.Tables, kqpCtx.Cluster, streamLookupIndex.Table().Path());
430429
const auto& [indexMeta, _] = tableDesc.Metadata->GetIndexMetadata(streamLookupIndex.Index().StringValue());
@@ -435,7 +434,7 @@ TExprBase KqpRewriteStreamLookupIndex(const TExprBase& node, TExprContext& ctx,
435434
.Table(BuildTableMeta(*indexMeta, node.Pos(), ctx))
436435
.LookupKeys(streamLookupIndex.LookupKeys())
437436
.Columns(streamLookupIndex.Columns())
438-
.Settings(settings.BuildNode(ctx, node.Pos()))
437+
.Settings(streamLookupIndex.Settings())
439438
.Done();
440439
}
441440

@@ -445,7 +444,7 @@ TExprBase KqpRewriteStreamLookupIndex(const TExprBase& node, TExprContext& ctx,
445444
.Table(BuildTableMeta(*indexMeta, node.Pos(), ctx))
446445
.LookupKeys(streamLookupIndex.LookupKeys())
447446
.Columns(keyColumnsList)
448-
.Settings(settings.BuildNode(ctx, node.Pos()))
447+
.Settings(streamLookupIndex.Settings())
449448
.Done();
450449

451450
TMaybeNode<TExprBase> lookupKeys;
@@ -473,6 +472,9 @@ TExprBase KqpRewriteStreamLookupIndex(const TExprBase& node, TExprContext& ctx,
473472
lookupKeys = lookupIndexTable;
474473
}
475474

475+
// We should allow lookup by null keys here,
476+
// because main table pk can contain nulls and we don't want to lose these rows
477+
settings.AllowNullKeysPrefixSize = keyColumnsList.Size();
476478
return Build<TKqlStreamLookupTable>(ctx, node.Pos())
477479
.Table(streamLookupIndex.Table())
478480
.LookupKeys(lookupKeys.Cast())

ydb/core/kqp/opt/logical/kqp_opt_log_join.cpp

Lines changed: 31 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -404,6 +404,7 @@ TMaybeNode<TExprBase> BuildKqpStreamIndexLookupJoin(
404404
}
405405

406406
TKqpStreamLookupSettings settings;
407+
settings.AllowNullKeysPrefixSize = rightLookup.PrefixSize;
407408
settings.Strategy = join.JoinType().Value() == "LeftSemi"
408409
? EStreamLookupStrategyType::LookupSemiJoinRows
409410
: EStreamLookupStrategyType::LookupJoinRows;
@@ -616,8 +617,37 @@ TMaybeNode<TExprBase> KqpJoinToIndexLookupImpl(const TDqJoin& join, TExprContext
616617
leftJoinKeys.emplace(leftKey);
617618
}
618619

620+
auto rightPKContainsAllJoinKeyColumns = [&](const auto& rightKeyColumns, const auto& rightJoinKeyColumns) {
621+
ui32 lookupPrefixSize = 0;
622+
TSet<TString> lookupKeyColumns;
623+
for (const auto& rightKeyColumn : rightKeyColumns) {
624+
auto leftColumn = rightJoinKeyToLeft.FindPtr(rightKeyColumn);
625+
626+
if (lookupPrefixSize < rightPrefixSize) {
627+
++lookupPrefixSize;
628+
} else {
629+
if (!leftColumn) {
630+
break;
631+
}
632+
}
633+
634+
lookupKeyColumns.insert(rightKeyColumn);
635+
}
636+
637+
for (auto& rightJoinKeyColumn : rightJoinKeyColumns) {
638+
if (!lookupKeyColumns.contains(rightJoinKeyColumn.Value())) {
639+
// this join key is non-pk column
640+
return false;
641+
}
642+
}
643+
644+
return true;
645+
};
646+
619647
const bool useStreamIndexLookupJoin = (kqpCtx.IsDataQuery() || kqpCtx.IsGenericQuery())
620-
&& kqpCtx.Config->EnableKqpDataQueryStreamIdxLookupJoin;
648+
&& kqpCtx.Config->EnableKqpDataQueryStreamIdxLookupJoin
649+
// StreamIndexLookupJoin can't execute join using non-pk columns
650+
&& rightPKContainsAllJoinKeyColumns(rightTableDesc.Metadata->KeyColumnNames, rightKeyColumns);
621651

622652
auto leftRowArg = Build<TCoArgument>(ctx, join.Pos())
623653
.Name("leftRowArg")

ydb/core/kqp/query_compiler/kqp_query_compiler.cpp

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1381,8 +1381,10 @@ class TKqpQueryCompiler : public IKqpQueryCompiler {
13811381

13821382
auto settings = TKqpStreamLookupSettings::Parse(streamLookup);
13831383
streamLookupProto.SetLookupStrategy(GetStreamLookupStrategy(settings.Strategy));
1384-
streamLookupProto.SetAllowNullKeys(settings.AllowNullKeys);
13851384
streamLookupProto.SetKeepRowsOrder(Config->OrderPreservingLookupJoinEnabled());
1385+
if (settings.AllowNullKeysPrefixSize) {
1386+
streamLookupProto.SetAllowNullKeysPrefixSize(*settings.AllowNullKeysPrefixSize);
1387+
}
13861388

13871389
switch (streamLookupProto.GetLookupStrategy()) {
13881390
case NKqpProto::EStreamLookupStrategy::LOOKUP: {

ydb/core/kqp/runtime/kqp_stream_lookup_worker.cpp

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -543,13 +543,15 @@ class TKqpJoinRows : public TKqpStreamLookupWorker {
543543
}
544544

545545
auto isKeyAllowed = [&](const TOwnedCellVec& cellVec) {
546-
if (Settings.HasAllowNullKeys() && Settings.GetAllowNullKeys()) {
546+
auto allowNullKeysPrefixSize = Settings.HasAllowNullKeysPrefixSize() ? Settings.GetAllowNullKeysPrefixSize() : 0;
547+
if (allowNullKeysPrefixSize >= cellVec.size()) {
548+
// all lookup key components can contain null
547549
return true;
548550
}
549551

550-
// otherwise we can't use nulls as lookup keys
551-
for (const auto& cell : cellVec) {
552-
if (cell.IsNull()) {
552+
// otherwise we can use nulls only for first allowNullKeysPrefixSize key components
553+
for (size_t i = 0; i < cellVec.size(); ++i) {
554+
if (cellVec[i].IsNull() && i >= allowNullKeysPrefixSize) {
553555
return false;
554556
}
555557
}

ydb/core/kqp/ut/indexes/kqp_indexes_ut.cpp

Lines changed: 193 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5444,6 +5444,199 @@ R"([[#;#;["Primary1"];[41u]];[["Secondary2"];[2u];["Primary2"];[42u]];[["Seconda
54445444
UNIT_ASSERT_VALUES_EQUAL(NYdb::FormatResultSetYson(result.GetResultSet(0)), R"([[[100u];[101u]];[[200u];[201u]]])");
54455445
}
54465446
}
5447+
5448+
Y_UNIT_TEST_TWIN(JoinWithNonPKColumnsInPredicate, UseStreamJoin) {
5449+
NKikimrConfig::TAppConfig appConfig;
5450+
appConfig.MutableTableServiceConfig()->SetEnableKqpDataQueryStreamIdxLookupJoin(UseStreamJoin);
5451+
auto setting = NKikimrKqp::TKqpSetting();
5452+
auto serverSettings = TKikimrSettings()
5453+
.SetKqpSettings({setting})
5454+
.SetAppConfig(appConfig);
5455+
TKikimrRunner kikimr(serverSettings);
5456+
auto db = kikimr.GetTableClient();
5457+
auto session = db.CreateSession().GetValueSync().GetSession();
5458+
5459+
{ // create tables
5460+
const TString createTableSql = R"(
5461+
CREATE TABLE `Root/tab1` (
5462+
id Text NOT NULL,
5463+
dst_ident Text,
5464+
exec_dt Timestamp,
5465+
PRIMARY KEY(id),
5466+
INDEX ix_exec_dt GLOBAL ON (exec_dt)
5467+
);
5468+
CREATE TABLE `Root/tab2` (
5469+
id Text NOT NULL,
5470+
int_ref Text,
5471+
ext_ref Text,
5472+
send_dttm Timestamp,
5473+
PRIMARY KEY(id),
5474+
INDEX ix_int_ref GLOBAL ON (int_ref),
5475+
INDEX ix_ext_ref GLOBAL ON (ext_ref),
5476+
INDEX ix_send_dttm GLOBAL ON (send_dttm)
5477+
);
5478+
5479+
CREATE TABLE `Root/tab3` (
5480+
id Text NOT NULL,
5481+
dst_ident Text,
5482+
exec_dt Timestamp,
5483+
PRIMARY KEY(id),
5484+
INDEX ix_exec_dt GLOBAL ON (exec_dt)
5485+
);
5486+
5487+
CREATE TABLE `Root/tab4` (
5488+
id Text NOT NULL,
5489+
int_ref Text,
5490+
ext_ref Text,
5491+
good_sign Text,
5492+
send_dttm Timestamp,
5493+
PRIMARY KEY(id),
5494+
INDEX ix_int_ref GLOBAL ON (int_ref),
5495+
INDEX ix_ext_ref GLOBAL ON (ext_ref),
5496+
INDEX ix_send_dttm GLOBAL ON (send_dttm)
5497+
);
5498+
5499+
CREATE TABLE `Root/tab5` (
5500+
t5_id Text NOT NULL,
5501+
t5_coll Text,
5502+
t5_exec_dt Timestamp,
5503+
PRIMARY KEY(t5_id),
5504+
INDEX ix_exec_dt GLOBAL ON (t5_exec_dt),
5505+
INDEX ix_ref_coll GLOBAL ON (t5_coll)
5506+
);
5507+
5508+
CREATE TABLE `Root/tab6` (
5509+
t6_id Text NOT NULL,
5510+
t6_coll Text NOT NULL,
5511+
t6_link_type Text,
5512+
PRIMARY KEY(t6_id, t6_coll),
5513+
INDEX ix_magic GLOBAL ON (t6_id, t6_link_type)
5514+
);
5515+
)";
5516+
5517+
auto result = session.ExecuteSchemeQuery(createTableSql).GetValueSync();
5518+
UNIT_ASSERT_C(result.GetIssues().Empty(), result.GetIssues().ToString());
5519+
UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS);
5520+
}
5521+
5522+
{ // fill tables
5523+
const TString upsertSql(Q_(R"(
5524+
UPSERT INTO `Root/tab1` (id, dst_ident, exec_dt) VALUES
5525+
('t1-0'u, 'id-1'u, Timestamp('2024-12-03T01:00:00.000000Z')),
5526+
('t1-1'u, 'id-2'u, Timestamp('2024-12-03T02:00:00.000000Z')),
5527+
('t1-2'u, 'id-3'u, Timestamp('2024-12-03T03:00:00.000000Z'));
5528+
5529+
UPSERT INTO `Root/tab2` (id, int_ref, ext_ref, send_dttm) VALUES
5530+
('t2-0'u, 't1-0'u, 'id-1'u, Timestamp('2024-12-03T01:01:00.000000Z')),
5531+
('t2-1'u, 't1-0'u, null, Timestamp('2024-12-03T02:01:00.000000Z')),
5532+
('t2-2'u, 't1-0'u, null, Timestamp('2024-12-03T02:01:00.000000Z')),
5533+
('t2-3'u, 't1-1'u, 'id-2'u, Timestamp('2024-12-03T03:01:00.000000Z')),
5534+
('t2-4'u, 't1-1'u, null, Timestamp('2024-12-03T04:01:00.000000Z')),
5535+
('t2-5'u, 't1-1'u, null, Timestamp('2024-12-03T05:01:00.000000Z')),
5536+
('t2-6'u, 't1-2'u, 'id-3'u, Timestamp('2024-12-03T06:01:00.000000Z')),
5537+
('t2-7'u, 't1-2'u, null, Timestamp('2024-12-03T07:01:00.000000Z')),
5538+
('t2-8'u, 't1-2'u, null, Timestamp('2024-12-03T08:01:00.000000Z')),
5539+
('t2-9'u, 't1-2'u, null, Timestamp('2024-12-03T09:01:00.000000Z'));
5540+
5541+
UPSERT INTO `Root/tab3` (id, dst_ident, exec_dt) VALUES
5542+
('t1-0'u, 'id-1'u, Timestamp('2024-12-03T01:00:00.000000Z')),
5543+
('t1-1'u, 'id-2'u, Timestamp('2024-12-03T02:00:00.000000Z')),
5544+
('t1-2'u, 'id-3'u, Timestamp('2024-12-03T03:00:00.000000Z'));
5545+
5546+
UPSERT INTO `Root/tab4` (id, int_ref, ext_ref, good_sign, send_dttm) VALUES
5547+
('t2-0'u, 't1-0'u, 'id-1'u, 'GOOD'u, Timestamp('2024-12-03T01:01:00.000000Z')),
5548+
('t2-1'u, 't1-0'u, null, 'BAD'u, Timestamp('2024-12-03T02:01:00.000000Z')),
5549+
('t2-2'u, 't1-0'u, null, 'BAD'u, Timestamp('2024-12-03T02:01:00.000000Z')),
5550+
('t2-3'u, 't1-1'u, 'id-2'u, 'GOOD'u, Timestamp('2024-12-03T03:01:00.000000Z')),
5551+
('t2-4'u, 't1-1'u, null, 'BAD'u, Timestamp('2024-12-03T04:01:00.000000Z')),
5552+
('t2-5'u, 't1-1'u, null, 'BAD'u, Timestamp('2024-12-03T05:01:00.000000Z')),
5553+
('t2-6'u, 't1-2'u, 'id-3'u, 'GOOD'u, Timestamp('2024-12-03T06:01:00.000000Z')),
5554+
('t2-7'u, 't1-2'u, null, 'BAD'u, Timestamp('2024-12-03T07:01:00.000000Z')),
5555+
('t2-8'u, 't1-2'u, null, 'BAD'u, Timestamp('2024-12-03T08:01:00.000000Z')),
5556+
('t2-9'u, 't1-2'u, null, 'BAD'u, Timestamp('2024-12-03T09:01:00.000000Z'));
5557+
5558+
UPSERT INTO `Root/tab5` (t5_id, t5_coll, t5_exec_dt) VALUES
5559+
('k00'u, null, Timestamp('2024-12-03T01:00:00.000000Z')),
5560+
('k01'u, null, Timestamp('2024-12-03T02:00:00.000000Z')),
5561+
('k02'u, null, Timestamp('2024-12-03T03:00:00.000000Z')),
5562+
('k10'u, 'c00'u, Timestamp('2024-12-03T08:00:00.000000Z'));
5563+
5564+
UPSERT INTO `Root/tab6` (t6_id, t6_coll, t6_link_type) VALUES
5565+
('k00'u, 'c00'u, 'l00'u),
5566+
('k01'u, 'c00'u, 'l00'u),
5567+
('k02'u, 'c00'u, 'l00'u);
5568+
)"));
5569+
5570+
auto result = session.ExecuteDataQuery(upsertSql, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).ExtractValueSync();
5571+
UNIT_ASSERT(result.IsSuccess());
5572+
}
5573+
5574+
{ // join with data column req.ext_ref in predicate
5575+
const TString joinSql(Q1_(R"(
5576+
$sys_date = DateTime("2024-12-03T00:00:00Z");
5577+
$ival = DateTime::IntervalFromDays(1);
5578+
SELECT req.id, doc.id, req.int_ref, doc.dst_ident, req.ext_ref
5579+
FROM `Root/tab1` VIEW ix_exec_dt AS doc
5580+
LEFT JOIN `Root/tab2` VIEW ix_int_ref req
5581+
ON doc.id = req.int_ref AND doc.dst_ident = req.ext_ref
5582+
WHERE doc.exec_dt >= $sys_date and doc.exec_dt <$sys_date + $ival
5583+
AND doc.id='t1-1'u;
5584+
)"));
5585+
5586+
auto result = session.ExecuteDataQuery(joinSql, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).ExtractValueSync();
5587+
UNIT_ASSERT(result.IsSuccess());
5588+
CompareYson(R"([
5589+
[["t2-3"];"t1-1";["t1-1"];["id-2"];["id-2"]]
5590+
])", FormatResultSetYson(result.GetResultSet(0)));
5591+
}
5592+
5593+
{ // join with data column req.ext_ref in predicate
5594+
const TString joinSql(Q1_(R"(
5595+
$sys_date = DateTime("2024-12-03T00:00:00Z");
5596+
$ival = DateTime::IntervalFromDays(1);
5597+
SELECT doc.id, req.id, doc.dst_ident, req.ext_ref, req.good_sign
5598+
FROM `Root/tab3` VIEW ix_exec_dt AS doc
5599+
LEFT JOIN (
5600+
SELECT id, int_ref, ext_ref, good_sign
5601+
FROM `Root/tab4` VIEW ix_int_ref
5602+
WHERE good_sign='GOOD'u
5603+
) AS req
5604+
ON doc.id = req.int_ref AND doc.dst_ident = req.ext_ref
5605+
WHERE doc.exec_dt >= $sys_date and doc.exec_dt <$sys_date + $ival
5606+
AND doc.id='t1-1'u;
5607+
)"));
5608+
5609+
auto result = session.ExecuteDataQuery(joinSql, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).ExtractValueSync();
5610+
UNIT_ASSERT(result.IsSuccess());
5611+
CompareYson(R"([
5612+
["t1-1";["t2-3"];["id-2"];["id-2"];["GOOD"]]
5613+
])", FormatResultSetYson(result.GetResultSet(0)));
5614+
}
5615+
5616+
{ // join with secondary index column in predicate
5617+
const TString joinSql(Q1_(R"(
5618+
SELECT * FROM (
5619+
SELECT t5.*, t6.*, t5owner.t5_id as owner_id
5620+
FROM (SELECT 'l00'u AS link_type) AS cond1
5621+
CROSS JOIN `Root/tab5` AS t5
5622+
LEFT JOIN `Root/tab6` VIEW ix_magic AS t6
5623+
ON t5.t5_id=t6.t6_id AND t6.t6_link_type=cond1.link_type
5624+
LEFT JOIN `Root/tab5` VIEW ix_ref_coll AS t5owner
5625+
ON t6.t6_coll=t5owner.t5_coll
5626+
) WHERE t5_exec_dt BETWEEN DateTime('2024-12-03T00:00:00Z') AND DateTime('2024-12-05T00:00:00Z')
5627+
ORDER BY t5_id, t6_id, owner_id;
5628+
)"));
5629+
5630+
auto result = session.ExecuteDataQuery(joinSql, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).ExtractValueSync();
5631+
UNIT_ASSERT(result.IsSuccess());
5632+
CompareYson(R"([
5633+
[["k10"];#;[1733187600000000u];"k00";["c00"];["k00"];["l00"]];
5634+
[["k10"];#;[1733191200000000u];"k01";["c00"];["k01"];["l00"]];
5635+
[["k10"];#;[1733194800000000u];"k02";["c00"];["k02"];["l00"]];
5636+
[#;["c00"];[1733212800000000u];"k10";#;#;#]
5637+
])", FormatResultSetYson(result.GetResultSet(0)));
5638+
}
5639+
}
54475640
}
54485641

54495642
}

ydb/core/protos/kqp.proto

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -750,6 +750,7 @@ message TKqpStreamLookupSettings {
750750
optional uint32 LockNodeId = 10;
751751
optional bool KeepRowsOrder = 11;
752752
optional bool AllowNullKeys = 12;
753+
optional uint32 AllowNullKeysPrefixSize = 13;
753754
}
754755

755756
message TKqpSequencerSettings {

ydb/core/protos/kqp_physical.proto

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -289,6 +289,7 @@ message TKqpPhyCnStreamLookup {
289289
EStreamLookupStrategy LookupStrategy = 6;
290290
bool KeepRowsOrder = 7;
291291
bool AllowNullKeys = 8;
292+
uint32 AllowNullKeysPrefixSize = 9;
292293
}
293294

294295
message TKqpPhyCnSequencer {

0 commit comments

Comments
 (0)